diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index a5aeef143e35..684debd9fea1 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -43,7 +43,7 @@ jobs: - name: Test Server RPM run: | export MAVEN_OPTS="${MAVEN_INSTALL_OPTS}" - $RETRY ./mvnw verify -B -P ci -pl :trino-server-rpm + $RETRY bash -c './mvnw verify -B -P ci -pl :trino-server-rpm || find core/trino-server-rpm/ -exec ls -ald {} +' - name: Clean Maven Output run: ./mvnw clean -pl '!:trino-server,!:trino-cli' - name: Test Docker Image @@ -245,7 +245,7 @@ jobs: - ":trino-oracle" - ":trino-kudu" - ":trino-iceberg,:trino-druid" - - ":trino-phoenix,:trino-phoenix5" + # TODO (https://github.com/trinodb/trino/issues/7534) - ":trino-phoenix,:trino-phoenix5" timeout-minutes: 60 steps: - uses: actions/checkout@v2 diff --git a/client/trino-cli/pom.xml b/client/trino-cli/pom.xml index 85089123041b..a9937a4548f1 100644 --- a/client/trino-cli/pom.xml +++ b/client/trino-cli/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/client/trino-cli/src/main/java/io/trino/cli/QueryRunner.java b/client/trino-cli/src/main/java/io/trino/cli/QueryRunner.java index 4de030dca5a7..8160fab605a3 100644 --- a/client/trino-cli/src/main/java/io/trino/cli/QueryRunner.java +++ b/client/trino-cli/src/main/java/io/trino/cli/QueryRunner.java @@ -20,6 +20,7 @@ import io.trino.client.StatementClient; import io.trino.client.auth.external.ExternalAuthenticator; import io.trino.client.auth.external.HttpTokenPoller; +import io.trino.client.auth.external.KnownToken; import io.trino.client.auth.external.RedirectHandler; import io.trino.client.auth.external.TokenPoller; import okhttp3.OkHttpClient; @@ -34,7 +35,6 @@ import static com.google.common.base.Preconditions.checkArgument; import static io.trino.client.ClientSession.stripTransactionId; import static io.trino.client.OkHttpUtil.basicAuth; -import static io.trino.client.OkHttpUtil.setupChannelSocket; import static io.trino.client.OkHttpUtil.setupCookieJar; import static io.trino.client.OkHttpUtil.setupHttpProxy; import static io.trino.client.OkHttpUtil.setupKerberos; @@ -93,7 +93,6 @@ public QueryRunner( OkHttpClient.Builder builder = new OkHttpClient.Builder(); - setupChannelSocket(builder); setupTimeouts(builder, 30, SECONDS); setupCookieJar(builder); setupSocksProxy(builder, socksProxy); @@ -195,6 +194,7 @@ private static void setupExternalAuth( ExternalAuthenticator authenticator = new ExternalAuthenticator( redirectHandler, poller, + KnownToken.local(), Duration.ofMinutes(10)); builder.authenticator(authenticator); diff --git a/client/trino-client/pom.xml b/client/trino-client/pom.xml index 8c4e4de10e52..62e612ab4b5e 100644 --- a/client/trino-client/pom.xml +++ b/client/trino-client/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml @@ -81,6 +81,12 @@ test + + io.airlift + concurrent + test + + com.squareup.okhttp3 mockwebserver diff --git a/client/trino-client/src/main/java/io/trino/client/OkHttpUtil.java b/client/trino-client/src/main/java/io/trino/client/OkHttpUtil.java index 9332b310119e..3ae798fd24b0 100644 --- a/client/trino-client/src/main/java/io/trino/client/OkHttpUtil.java +++ b/client/trino-client/src/main/java/io/trino/client/OkHttpUtil.java @@ -14,16 +14,12 @@ package io.trino.client; import com.google.common.base.CharMatcher; -import com.google.common.base.Splitter; -import com.google.common.base.StandardSystemProperty; -import com.google.common.collect.ImmutableList; import com.google.common.net.HostAndPort; import io.airlift.security.pem.PemReader; import okhttp3.Credentials; import okhttp3.Interceptor; import okhttp3.JavaNetCookieJar; import okhttp3.OkHttpClient; -import okhttp3.Protocol; import okhttp3.Request; import okhttp3.internal.tls.LegacyHostnameVerifier; @@ -246,26 +242,6 @@ public static void setupSsl( } } - public static void setupChannelSocket(OkHttpClient.Builder clientBuilder) - { - // Enable socket factory only for pre JDK 11 - if (!isAtLeastJava11()) { - clientBuilder.socketFactory(new SocketChannelSocketFactory()); - clientBuilder.protocols(ImmutableList.of(Protocol.HTTP_1_1)); - } - } - - private static boolean isAtLeastJava11() - { - String feature = Splitter.on(".").split(StandardSystemProperty.JAVA_VERSION.value()).iterator().next(); - try { - return Integer.parseInt(feature) >= 11; - } - catch (NumberFormatException e) { - return false; - } - } - private static void validateCertificates(KeyStore keyStore) throws GeneralSecurityException { diff --git a/client/trino-client/src/main/java/io/trino/client/SocketChannelSocketFactory.java b/client/trino-client/src/main/java/io/trino/client/SocketChannelSocketFactory.java deleted file mode 100644 index c52d5df79d11..000000000000 --- a/client/trino-client/src/main/java/io/trino/client/SocketChannelSocketFactory.java +++ /dev/null @@ -1,67 +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 io.trino.client; - -import javax.net.SocketFactory; - -import java.io.IOException; -import java.net.InetAddress; -import java.net.InetSocketAddress; -import java.net.Socket; -import java.net.SocketException; -import java.nio.channels.SocketChannel; - -/** - * Workaround for JDK IPv6 bug on Mac. Sockets created with the basic socket - * API often cannot connect to IPv6 destinations due to JDK-8131133. However, - * NIO sockets do not have this problem, even if used in blocking mode. - */ -public class SocketChannelSocketFactory - extends SocketFactory -{ - @Override - public Socket createSocket() - throws IOException - { - return SocketChannel.open().socket(); - } - - @Override - public Socket createSocket(String host, int port) - throws IOException - { - return SocketChannel.open(new InetSocketAddress(host, port)).socket(); - } - - @Override - public Socket createSocket(String host, int port, InetAddress localAddress, int localPort) - throws IOException - { - throw new SocketException("not supported"); - } - - @Override - public Socket createSocket(InetAddress address, int port) - throws IOException - { - return SocketChannel.open(new InetSocketAddress(address, port)).socket(); - } - - @Override - public Socket createSocket(InetAddress address, int port, InetAddress localAddress, int localPort) - throws IOException - { - throw new SocketException("not supported"); - } -} diff --git a/client/trino-client/src/main/java/io/trino/client/auth/external/ExternalAuthenticator.java b/client/trino-client/src/main/java/io/trino/client/auth/external/ExternalAuthenticator.java index 2269cf87c2ac..e91ff6572dcd 100644 --- a/client/trino-client/src/main/java/io/trino/client/auth/external/ExternalAuthenticator.java +++ b/client/trino-client/src/main/java/io/trino/client/auth/external/ExternalAuthenticator.java @@ -44,12 +44,13 @@ public class ExternalAuthenticator private final TokenPoller tokenPoller; private final RedirectHandler redirectHandler; private final Duration timeout; - private Token knownToken; + private final KnownToken knownToken; - public ExternalAuthenticator(RedirectHandler redirect, TokenPoller tokenPoller, Duration timeout) + public ExternalAuthenticator(RedirectHandler redirect, TokenPoller tokenPoller, KnownToken knownToken, Duration timeout) { this.tokenPoller = requireNonNull(tokenPoller, "tokenPoller is null"); this.redirectHandler = requireNonNull(redirect, "redirect is null"); + this.knownToken = requireNonNull(knownToken, "knownToken is null"); this.timeout = requireNonNull(timeout, "timeout is null"); } @@ -57,28 +58,27 @@ public ExternalAuthenticator(RedirectHandler redirect, TokenPoller tokenPoller, @Override public Request authenticate(Route route, Response response) { - knownToken = null; - - Optional authentication = toAuthentication(response); - if (!authentication.isPresent()) { - return null; - } + knownToken.setupToken(() -> { + Optional authentication = toAuthentication(response); + if (!authentication.isPresent()) { + return Optional.empty(); + } - Optional token = authentication.get().obtainToken(timeout, redirectHandler, tokenPoller); - if (!token.isPresent()) { - return null; - } + return authentication.get().obtainToken(timeout, redirectHandler, tokenPoller); + }); - knownToken = token.get(); - return withBearerToken(response.request(), knownToken); + return knownToken.getToken() + .map(token -> withBearerToken(response.request(), token)) + .orElse(null); } @Override public Response intercept(Chain chain) throws IOException { - if (knownToken != null) { - return chain.proceed(withBearerToken(chain.request(), knownToken)); + Optional token = knownToken.getToken(); + if (token.isPresent()) { + return chain.proceed(withBearerToken(chain.request(), token.get())); } return chain.proceed(chain.request()); diff --git a/client/trino-client/src/main/java/io/trino/client/auth/external/KnownToken.java b/client/trino-client/src/main/java/io/trino/client/auth/external/KnownToken.java new file mode 100644 index 000000000000..240af9075763 --- /dev/null +++ b/client/trino-client/src/main/java/io/trino/client/auth/external/KnownToken.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 io.trino.client.auth.external; + +import java.util.Optional; +import java.util.function.Supplier; + +public interface KnownToken +{ + Optional getToken(); + + void setupToken(Supplier> tokenSource); + + static KnownToken local() + { + return new LocalKnownToken(); + } + + static KnownToken memoryCached() + { + return MemoryCachedKnownToken.INSTANCE; + } +} diff --git a/client/trino-client/src/main/java/io/trino/client/auth/external/LocalKnownToken.java b/client/trino-client/src/main/java/io/trino/client/auth/external/LocalKnownToken.java new file mode 100644 index 000000000000..40b984a0764b --- /dev/null +++ b/client/trino-client/src/main/java/io/trino/client/auth/external/LocalKnownToken.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 io.trino.client.auth.external; + +import javax.annotation.concurrent.NotThreadSafe; + +import java.util.Optional; +import java.util.function.Supplier; + +import static java.util.Objects.requireNonNull; + +/** + * LocalKnownToken class keeps the token on its field + * and it's designed to use it in fully serialized manner. + */ +@NotThreadSafe +class LocalKnownToken + implements KnownToken +{ + private Optional knownToken = Optional.empty(); + + @Override + public Optional getToken() + { + return knownToken; + } + + @Override + public void setupToken(Supplier> tokenSource) + { + requireNonNull(tokenSource, "tokenSource is null"); + + knownToken = tokenSource.get(); + } +} diff --git a/client/trino-client/src/main/java/io/trino/client/auth/external/MemoryCachedKnownToken.java b/client/trino-client/src/main/java/io/trino/client/auth/external/MemoryCachedKnownToken.java new file mode 100644 index 000000000000..e8513e4bd87f --- /dev/null +++ b/client/trino-client/src/main/java/io/trino/client/auth/external/MemoryCachedKnownToken.java @@ -0,0 +1,83 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.client.auth.external; + +import javax.annotation.concurrent.ThreadSafe; + +import java.util.Optional; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.function.Supplier; + +/** + * This KnownToken instance forces all Connections to reuse same token. + * Every time an existing token is considered to be invalid each Connection + * will try to obtain a new token, but only the first one will actually do the job, + * where every other connection will be waiting on readLock + * until obtaining new token finishes. + *

+ * In general the game is to reuse same token and obtain it only once, no matter how + * many Connections will be actively using it. It's very important as obtaining the new token + * will take minutes, as it mostly requires user thinking time. + */ +@ThreadSafe +class MemoryCachedKnownToken + implements KnownToken +{ + public static final MemoryCachedKnownToken INSTANCE = new MemoryCachedKnownToken(); + + private final ReadWriteLock lock = new ReentrantReadWriteLock(); + private final Lock readLock = lock.readLock(); + private final Lock writeLock = lock.writeLock(); + private Optional knownToken = Optional.empty(); + + private MemoryCachedKnownToken() + { + } + + @Override + public Optional getToken() + { + try { + readLock.lockInterruptibly(); + return knownToken; + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + finally { + readLock.unlock(); + } + } + + @Override + public void setupToken(Supplier> tokenSource) + { + // Try to lock and generate new token. If some other thread (Connection) has + // already obtained writeLock and is generating new token, then skipp this + // to block on getToken() + if (writeLock.tryLock()) { + try { + // Clear knownToken before obtaining new token, as it might fail leaving old invalid token. + knownToken = Optional.empty(); + knownToken = tokenSource.get(); + } + finally { + writeLock.unlock(); + } + } + } +} diff --git a/client/trino-client/src/test/java/io/trino/client/TestServerInfo.java b/client/trino-client/src/test/java/io/trino/client/TestServerInfo.java index ca74648d4c0d..66390b162f0e 100644 --- a/client/trino-client/src/test/java/io/trino/client/TestServerInfo.java +++ b/client/trino-client/src/test/java/io/trino/client/TestServerInfo.java @@ -21,6 +21,7 @@ import static io.airlift.json.JsonCodec.jsonCodec; import static io.trino.client.NodeVersion.UNKNOWN; +import static java.util.concurrent.TimeUnit.MINUTES; import static org.testng.Assert.assertEquals; public class TestServerInfo @@ -30,7 +31,7 @@ public class TestServerInfo @Test public void testJsonRoundTrip() { - assertJsonRoundTrip(new ServerInfo(UNKNOWN, "test", true, false, Optional.of(Duration.valueOf("2m")))); + assertJsonRoundTrip(new ServerInfo(UNKNOWN, "test", true, false, Optional.of(new Duration(2, MINUTES)))); assertJsonRoundTrip(new ServerInfo(UNKNOWN, "test", true, false, Optional.empty())); } diff --git a/client/trino-client/src/test/java/io/trino/client/auth/external/MockRedirectHandler.java b/client/trino-client/src/test/java/io/trino/client/auth/external/MockRedirectHandler.java new file mode 100644 index 000000000000..cf671ea391c9 --- /dev/null +++ b/client/trino-client/src/test/java/io/trino/client/auth/external/MockRedirectHandler.java @@ -0,0 +1,59 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.trino.client.auth.external; + +import java.net.URI; +import java.time.Duration; +import java.util.concurrent.atomic.AtomicInteger; + +public class MockRedirectHandler + implements RedirectHandler +{ + private URI redirectedTo; + private AtomicInteger redirectionCount = new AtomicInteger(0); + private Duration redirectTime; + + @Override + public void redirectTo(URI uri) + throws RedirectException + { + redirectedTo = uri; + redirectionCount.incrementAndGet(); + try { + if (redirectTime != null) { + Thread.sleep(redirectTime.toMillis()); + } + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + } + + public URI redirectedTo() + { + return redirectedTo; + } + + public int getRedirectionCount() + { + return redirectionCount.get(); + } + + public MockRedirectHandler sleepOnRedirect(Duration redirectTime) + { + this.redirectTime = redirectTime; + return this; + } +} diff --git a/client/trino-client/src/test/java/io/trino/client/auth/external/MockTokenPoller.java b/client/trino-client/src/test/java/io/trino/client/auth/external/MockTokenPoller.java index fae34ea9c031..d07205f38636 100644 --- a/client/trino-client/src/test/java/io/trino/client/auth/external/MockTokenPoller.java +++ b/client/trino-client/src/test/java/io/trino/client/auth/external/MockTokenPoller.java @@ -17,21 +17,21 @@ import java.net.URI; import java.time.Duration; -import java.util.ArrayDeque; -import java.util.HashMap; import java.util.Map; -import java.util.Queue; +import java.util.concurrent.BlockingDeque; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.LinkedBlockingDeque; public final class MockTokenPoller implements TokenPoller { - private final Map> results = new HashMap<>(); + private final Map> results = new ConcurrentHashMap<>(); public MockTokenPoller withResult(URI tokenUri, TokenPollResult result) { results.compute(tokenUri, (uri, queue) -> { if (queue == null) { - return new ArrayDeque<>(ImmutableList.of(result)); + return new LinkedBlockingDeque<>(ImmutableList.of(result)); } queue.add(result); return queue; @@ -42,7 +42,7 @@ public MockTokenPoller withResult(URI tokenUri, TokenPollResult result) @Override public TokenPollResult pollForToken(URI tokenUri, Duration ignored) { - Queue queue = results.get(tokenUri); + BlockingDeque queue = results.get(tokenUri); if (queue == null) { throw new IllegalArgumentException("Unknown token URI: " + tokenUri); } diff --git a/client/trino-client/src/test/java/io/trino/client/auth/external/TestExternalAuthentication.java b/client/trino-client/src/test/java/io/trino/client/auth/external/TestExternalAuthentication.java index 279d1966b4c7..dce341d20392 100644 --- a/client/trino-client/src/test/java/io/trino/client/auth/external/TestExternalAuthentication.java +++ b/client/trino-client/src/test/java/io/trino/client/auth/external/TestExternalAuthentication.java @@ -125,22 +125,4 @@ public void testObtainTokenWhenNoRedirectUriHasBeenProvided() assertThat(redirectHandler.redirectedTo()).isNull(); assertThat(token).map(Token::token).hasValue(AUTH_TOKEN); } - - private static class MockRedirectHandler - implements RedirectHandler - { - private URI redirectedTo; - - @Override - public void redirectTo(URI uri) - throws RedirectException - { - redirectedTo = uri; - } - - public URI redirectedTo() - { - return redirectedTo; - } - } } diff --git a/client/trino-client/src/test/java/io/trino/client/auth/external/TestExternalAuthenticator.java b/client/trino-client/src/test/java/io/trino/client/auth/external/TestExternalAuthenticator.java index 729ffe6fa023..c9ea6fc73199 100644 --- a/client/trino-client/src/test/java/io/trino/client/auth/external/TestExternalAuthenticator.java +++ b/client/trino-client/src/test/java/io/trino/client/auth/external/TestExternalAuthenticator.java @@ -13,31 +13,55 @@ */ package io.trino.client.auth.external; +import com.google.common.collect.ImmutableList; import io.trino.client.ClientException; import okhttp3.HttpUrl; import okhttp3.Protocol; import okhttp3.Request; import okhttp3.Response; +import org.assertj.core.api.ListAssert; +import org.assertj.core.api.ThrowableAssert; +import org.testng.annotations.AfterClass; import org.testng.annotations.Test; import java.net.URI; import java.net.URISyntaxException; import java.time.Duration; +import java.util.ArrayList; +import java.util.List; import java.util.Optional; +import java.util.concurrent.Callable; +import java.util.concurrent.CancellationException; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.stream.Stream; +import static com.google.common.base.Preconditions.checkState; +import static com.google.common.collect.ImmutableList.toImmutableList; import static com.google.common.net.HttpHeaders.AUTHORIZATION; import static com.google.common.net.HttpHeaders.WWW_AUTHENTICATE; +import static io.airlift.concurrent.Threads.daemonThreadsNamed; import static io.trino.client.auth.external.ExternalAuthenticator.TOKEN_URI_FIELD; import static io.trino.client.auth.external.ExternalAuthenticator.toAuthentication; import static io.trino.client.auth.external.TokenPollResult.successful; import static java.lang.String.format; import static java.net.HttpURLConnection.HTTP_UNAUTHORIZED; import static java.net.URI.create; +import static java.util.concurrent.Executors.newCachedThreadPool; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; public class TestExternalAuthenticator { + private static final ExecutorService executor = newCachedThreadPool(daemonThreadsNamed(TestExternalAuthenticator.class.getName() + "-%d")); + + @AfterClass(alwaysRun = true) + public void shutDownThreadPool() + { + executor.shutdownNow(); + } + @Test public void testChallengeWithOnlyTokenServerUri() { @@ -110,7 +134,7 @@ public void testAuthentication() { MockTokenPoller tokenPoller = new MockTokenPoller() .withResult(URI.create("http://token.uri"), successful(new Token("valid-token"))); - ExternalAuthenticator authenticator = new ExternalAuthenticator(uri -> {}, tokenPoller, Duration.ofSeconds(1)); + ExternalAuthenticator authenticator = new ExternalAuthenticator(uri -> {}, tokenPoller, KnownToken.local(), Duration.ofSeconds(1)); Request authenticated = authenticator.authenticate(null, getUnauthorizedResponse("Bearer x_token_server=\"http://token.uri\"")); @@ -125,7 +149,7 @@ public void testReAuthenticationAfterRejectingToken() MockTokenPoller tokenPoller = new MockTokenPoller() .withResult(URI.create("http://token.uri"), successful(new Token("first-token"))) .withResult(URI.create("http://token.uri"), successful(new Token("second-token"))); - ExternalAuthenticator authenticator = new ExternalAuthenticator(uri -> {}, tokenPoller, Duration.ofSeconds(1)); + ExternalAuthenticator authenticator = new ExternalAuthenticator(uri -> {}, tokenPoller, KnownToken.local(), Duration.ofSeconds(1)); Request request = authenticator.authenticate(null, getUnauthorizedResponse("Bearer x_token_server=\"http://token.uri\"")); Request reAuthenticated = authenticator.authenticate(null, getUnauthorizedResponse("Bearer x_token_server=\"http://token.uri\"", request)); @@ -134,6 +158,140 @@ public void testReAuthenticationAfterRejectingToken() .containsExactly("Bearer second-token"); } + @Test(timeOut = 2000) + public void testAuthenticationFromMultipleThreadsWithLocallyStoredToken() + { + MockTokenPoller tokenPoller = new MockTokenPoller() + .withResult(URI.create("http://token.uri"), successful(new Token("valid-token-1"))) + .withResult(URI.create("http://token.uri"), successful(new Token("valid-token-2"))) + .withResult(URI.create("http://token.uri"), successful(new Token("valid-token-3"))) + .withResult(URI.create("http://token.uri"), successful(new Token("valid-token-4"))); + MockRedirectHandler redirectHandler = new MockRedirectHandler(); + + ExternalAuthenticator authenticator = new ExternalAuthenticator(redirectHandler, tokenPoller, KnownToken.local(), Duration.ofSeconds(1)); + List> requests = times( + 4, + () -> authenticator.authenticate(null, getUnauthorizedResponse("Bearer x_token_server=\"http://token.uri\", x_redirect_server=\"http://redirect.uri\""))) + .map(executor::submit) + .collect(toImmutableList()); + + ConcurrentRequestAssertion assertion = new ConcurrentRequestAssertion(requests); + assertion.requests() + .extracting(Request::headers) + .extracting(headers -> headers.get(AUTHORIZATION)) + .contains("Bearer valid-token-1", "Bearer valid-token-2", "Bearer valid-token-3", "Bearer valid-token-4"); + assertion.assertThatNoExceptionsHasBeenThrown(); + assertThat(redirectHandler.getRedirectionCount()).isEqualTo(4); + } + + @Test(timeOut = 2000) + public void testAuthenticationFromMultipleThreadsWithCachedToken() + { + ExecutorService executor = newCachedThreadPool(daemonThreadsNamed(this.getClass().getName() + "%n")); + MockTokenPoller tokenPoller = new MockTokenPoller() + .withResult(URI.create("http://token.uri"), successful(new Token("valid-token"))); + MockRedirectHandler redirectHandler = new MockRedirectHandler() + .sleepOnRedirect(Duration.ofMillis(10)); + + ExternalAuthenticator authenticator = new ExternalAuthenticator(redirectHandler, tokenPoller, KnownToken.memoryCached(), Duration.ofSeconds(1)); + List> requests = times( + 4, + () -> authenticator.authenticate(null, getUnauthorizedResponse("Bearer x_token_server=\"http://token.uri\", x_redirect_server=\"http://redirect.uri\""))) + .map(executor::submit) + .collect(toImmutableList()); + + ConcurrentRequestAssertion assertion = new ConcurrentRequestAssertion(requests); + assertion.requests() + .extracting(Request::headers) + .extracting(headers -> headers.get(AUTHORIZATION)) + .containsOnly("Bearer valid-token"); + assertion.assertThatNoExceptionsHasBeenThrown(); + assertThat(redirectHandler.getRedirectionCount()).isEqualTo(1); + } + + @Test(timeOut = 2000) + public void testAuthenticationFromMultipleThreadsWithCachedTokenAfterAuthenticateFails() + { + MockTokenPoller tokenPoller = new MockTokenPoller() + .withResult(URI.create("http://token.uri"), TokenPollResult.successful(new Token("first-token"))) + .withResult(URI.create("http://token.uri"), TokenPollResult.failed("external authentication error")); + MockRedirectHandler redirectHandler = new MockRedirectHandler() + .sleepOnRedirect(Duration.ofMillis(10)); + + ExternalAuthenticator authenticator = new ExternalAuthenticator(redirectHandler, tokenPoller, KnownToken.memoryCached(), Duration.ofSeconds(1)); + Request firstRequest = authenticator.authenticate(null, getUnauthorizedResponse("Bearer x_token_server=\"http://token.uri\", x_redirect_server=\"http://redirect.uri\"")); + + List> requests = times( + 4, + () -> authenticator.authenticate(null, getUnauthorizedResponse("Bearer x_token_server=\"http://token.uri\", x_redirect_server=\"http://redirect.uri\"", firstRequest))) + .map(executor::submit) + .collect(toImmutableList()); + + ConcurrentRequestAssertion assertion = new ConcurrentRequestAssertion(requests); + assertion.requests().containsExactly(null, null, null); + assertion.firstException().hasMessage("external authentication error") + .isInstanceOf(ClientException.class); + + assertThat(redirectHandler.getRedirectionCount()).isEqualTo(2); + } + + @Test(timeOut = 2000) + public void testAuthenticationFromMultipleThreadsWithCachedTokenAfterAuthenticateTimesOut() + { + MockRedirectHandler redirectHandler = new MockRedirectHandler() + .sleepOnRedirect(Duration.ofMillis(5)); + + ExternalAuthenticator authenticator = new ExternalAuthenticator(redirectHandler, (uri, duration) -> TokenPollResult.pending(uri), KnownToken.memoryCached(), Duration.ofMillis(1)); + List> requests = times( + 4, + () -> authenticator.authenticate(null, getUnauthorizedResponse("Bearer x_token_server=\"http://token.uri\", x_redirect_server=\"http://redirect.uri\""))) + .map(executor::submit) + .collect(toImmutableList()); + + ConcurrentRequestAssertion assertion = new ConcurrentRequestAssertion(requests); + assertion.requests() + .containsExactly(null, null, null, null); + assertion.assertThatNoExceptionsHasBeenThrown(); + assertThat(redirectHandler.getRedirectionCount()).isEqualTo(1); + } + + @Test(timeOut = 2000) + public void testAuthenticationFromMultipleThreadsWithCachedTokenAfterAuthenticateIsInterrupted() + throws Exception + { + ExecutorService interruptableThreadPool = newCachedThreadPool(daemonThreadsNamed(this.getClass().getName() + "-interruptable-%d")); + MockRedirectHandler redirectHandler = new MockRedirectHandler() + .sleepOnRedirect(Duration.ofMinutes(1)); + + ExternalAuthenticator authenticator = new ExternalAuthenticator(redirectHandler, (uri, duration) -> TokenPollResult.pending(uri), KnownToken.memoryCached(), Duration.ofMillis(1)); + Future interruptedAuthentication = interruptableThreadPool.submit( + () -> authenticator.authenticate(null, getUnauthorizedResponse("Bearer x_token_server=\"http://token.uri\", x_redirect_server=\"http://redirect.uri\""))); + Thread.sleep(100); //It's here to make sure that authentication will start before the other threads. + List> requests = times( + 2, + () -> authenticator.authenticate(null, getUnauthorizedResponse("Bearer x_token_server=\"http://token.uri\", x_redirect_server=\"http://redirect.uri\""))) + .map(executor::submit) + .collect(toImmutableList()); + + Thread.sleep(100); + interruptableThreadPool.shutdownNow(); + + ConcurrentRequestAssertion assertion = new ConcurrentRequestAssertion(ImmutableList.>builder() + .addAll(requests) + .add(interruptedAuthentication) + .build()); + assertion.requests().containsExactly(null, null); + assertion.firstException().hasRootCauseInstanceOf(InterruptedException.class); + + assertThat(redirectHandler.getRedirectionCount()).isEqualTo(1); + } + + private static Stream> times(int times, Callable request) + { + return Stream.generate(() -> request) + .limit(times); + } + private static Optional buildAuthentication(String challengeHeader) { return toAuthentication(getUnauthorizedResponse(challengeHeader)); @@ -157,4 +315,50 @@ private static Response getUnauthorizedResponse(String challengeHeader, Request .header(WWW_AUTHENTICATE, challengeHeader) .build(); } + + static class ConcurrentRequestAssertion + { + private final List exceptions = new ArrayList<>(); + private final List requests = new ArrayList<>(); + + public ConcurrentRequestAssertion(List> requests) + { + for (Future request : requests) { + try { + this.requests.add(request.get()); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + catch (CancellationException ex) { + exceptions.add(ex); + } + catch (ExecutionException ex) { + checkState(ex.getCause() != null, "Missing cause on ExecutionException " + ex.getMessage()); + + exceptions.add(ex.getCause()); + } + } + } + + ThrowableAssert firstException() + { + return exceptions.stream() + .findFirst() + .map(ThrowableAssert::new) + .orElseGet(() -> new ThrowableAssert(() -> null)); + } + + void assertThatNoExceptionsHasBeenThrown() + { + assertThat(exceptions) + .isEmpty(); + } + + ListAssert requests() + { + return assertThat(requests); + } + } } diff --git a/client/trino-jdbc/pom.xml b/client/trino-jdbc/pom.xml index 74692b96692c..2fd3bbb69af6 100644 --- a/client/trino-jdbc/pom.xml +++ b/client/trino-jdbc/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/client/trino-jdbc/src/main/java/io/trino/jdbc/ConnectionProperties.java b/client/trino-jdbc/src/main/java/io/trino/jdbc/ConnectionProperties.java index c3db9662374b..5618e47bd1b4 100644 --- a/client/trino-jdbc/src/main/java/io/trino/jdbc/ConnectionProperties.java +++ b/client/trino-jdbc/src/main/java/io/trino/jdbc/ConnectionProperties.java @@ -55,6 +55,7 @@ enum SslVerificationMode public static final ConnectionProperty HTTP_PROXY = new HttpProxy(); public static final ConnectionProperty APPLICATION_NAME_PREFIX = new ApplicationNamePrefix(); public static final ConnectionProperty DISABLE_COMPRESSION = new DisableCompression(); + public static final ConnectionProperty ASSUME_LITERAL_NAMES_IN_METADATA_CALLS_FOR_NON_CONFORMING_CLIENTS = new AssumeLiteralNamesInMetadataCallsForNonConformingClients(); public static final ConnectionProperty SSL = new Ssl(); public static final ConnectionProperty SSL_VERIFICATION = new SslVerification(); public static final ConnectionProperty SSL_KEY_STORE_PATH = new SslKeyStorePath(); @@ -73,6 +74,7 @@ enum SslVerificationMode public static final ConnectionProperty ACCESS_TOKEN = new AccessToken(); public static final ConnectionProperty EXTERNAL_AUTHENTICATION = new ExternalAuthentication(); public static final ConnectionProperty EXTERNAL_AUTHENTICATION_TIMEOUT = new ExternalAuthenticationTimeout(); + public static final ConnectionProperty EXTERNAL_AUTHENTICATION_TOKEN_CACHE = new ExternalAuthenticationTokenCache(); public static final ConnectionProperty> EXTRA_CREDENTIALS = new ExtraCredentials(); public static final ConnectionProperty CLIENT_INFO = new ClientInfo(); public static final ConnectionProperty CLIENT_TAGS = new ClientTags(); @@ -89,6 +91,7 @@ enum SslVerificationMode .add(HTTP_PROXY) .add(APPLICATION_NAME_PREFIX) .add(DISABLE_COMPRESSION) + .add(ASSUME_LITERAL_NAMES_IN_METADATA_CALLS_FOR_NON_CONFORMING_CLIENTS) .add(SSL) .add(SSL_VERIFICATION) .add(SSL_KEY_STORE_PATH) @@ -113,6 +116,7 @@ enum SslVerificationMode .add(SOURCE) .add(EXTERNAL_AUTHENTICATION) .add(EXTERNAL_AUTHENTICATION_TIMEOUT) + .add(EXTERNAL_AUTHENTICATION_TOKEN_CACHE) .build(); private static final Map> KEY_LOOKUP = unmodifiableMap(ALL_PROPERTIES.stream() @@ -273,6 +277,15 @@ public DisableCompression() } } + private static class AssumeLiteralNamesInMetadataCallsForNonConformingClients + extends AbstractConnectionProperty + { + public AssumeLiteralNamesInMetadataCallsForNonConformingClients() + { + super("assumeLiteralNamesInMetadataCallsForNonConformingClients", NOT_REQUIRED, ALLOWED, BOOLEAN_CONVERTER); + } + } + private static class Ssl extends AbstractConnectionProperty { @@ -461,6 +474,15 @@ public ExternalAuthenticationTimeout() } } + private static class ExternalAuthenticationTokenCache + extends AbstractConnectionProperty + { + public ExternalAuthenticationTokenCache() + { + super("externalAuthenticationTokenCache", Optional.of(KnownTokenCache.NONE.name()), NOT_REQUIRED, ALLOWED, KnownTokenCache::valueOf); + } + } + private static class ExtraCredentials extends AbstractConnectionProperty> { diff --git a/client/trino-jdbc/src/main/java/io/trino/jdbc/KnownTokenCache.java b/client/trino-jdbc/src/main/java/io/trino/jdbc/KnownTokenCache.java new file mode 100644 index 000000000000..6c3dde57d8c7 --- /dev/null +++ b/client/trino-jdbc/src/main/java/io/trino/jdbc/KnownTokenCache.java @@ -0,0 +1,36 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.trino.jdbc; + +import io.trino.client.auth.external.KnownToken; + +public enum KnownTokenCache +{ + NONE { + @Override + KnownToken create() + { + return KnownToken.local(); + } + }, + MEMORY { + @Override + KnownToken create() + { + return KnownToken.memoryCached(); + } + }; + + abstract KnownToken create(); +} diff --git a/client/trino-jdbc/src/main/java/io/trino/jdbc/NonRegisteringTrinoDriver.java b/client/trino-jdbc/src/main/java/io/trino/jdbc/NonRegisteringTrinoDriver.java index cb63d2a18ac9..f529737efe2f 100644 --- a/client/trino-jdbc/src/main/java/io/trino/jdbc/NonRegisteringTrinoDriver.java +++ b/client/trino-jdbc/src/main/java/io/trino/jdbc/NonRegisteringTrinoDriver.java @@ -24,7 +24,6 @@ import java.util.Properties; import java.util.logging.Logger; -import static io.trino.client.OkHttpUtil.setupChannelSocket; import static io.trino.client.OkHttpUtil.userAgent; import static io.trino.jdbc.DriverInfo.DRIVER_NAME; import static io.trino.jdbc.DriverInfo.DRIVER_VERSION; @@ -112,9 +111,6 @@ private static OkHttpClient newHttpClient() { OkHttpClient.Builder builder = new OkHttpClient.Builder() .addInterceptor(userAgent(DRIVER_NAME + "/" + DRIVER_VERSION)); - - // Enable socket factory only for pre JDK 11 - setupChannelSocket(builder); return builder.build(); } } diff --git a/client/trino-jdbc/src/main/java/io/trino/jdbc/TrinoConnection.java b/client/trino-jdbc/src/main/java/io/trino/jdbc/TrinoConnection.java index 81c2b8cc148a..bea84728d960 100644 --- a/client/trino-jdbc/src/main/java/io/trino/jdbc/TrinoConnection.java +++ b/client/trino-jdbc/src/main/java/io/trino/jdbc/TrinoConnection.java @@ -91,6 +91,7 @@ public class TrinoConnection private final String user; private final Optional sessionUser; private final boolean compressionDisabled; + private final boolean assumeLiteralNamesInMetadataCallsForNonConformingClients; private final Map extraCredentials; private final Optional applicationNamePrefix; private final Optional source; @@ -115,6 +116,7 @@ public class TrinoConnection this.source = uri.getSource(); this.extraCredentials = uri.getExtraCredentials(); this.compressionDisabled = uri.isCompressionDisabled(); + this.assumeLiteralNamesInMetadataCallsForNonConformingClients = uri.isAssumeLiteralNamesInMetadataCallsForNonConformingClients(); this.queryExecutor = requireNonNull(queryExecutor, "queryExecutor is null"); uri.getClientInfo().ifPresent(tags -> clientInfo.put(CLIENT_INFO, tags)); uri.getClientTags().ifPresent(tags -> clientInfo.put(CLIENT_TAGS, tags)); @@ -238,7 +240,7 @@ public boolean isClosed() public DatabaseMetaData getMetaData() throws SQLException { - return new TrinoDatabaseMetaData(this); + return new TrinoDatabaseMetaData(this, assumeLiteralNamesInMetadataCallsForNonConformingClients); } @Override diff --git a/client/trino-jdbc/src/main/java/io/trino/jdbc/TrinoDatabaseMetaData.java b/client/trino-jdbc/src/main/java/io/trino/jdbc/TrinoDatabaseMetaData.java index 49033d3ca876..9782234da699 100644 --- a/client/trino-jdbc/src/main/java/io/trino/jdbc/TrinoDatabaseMetaData.java +++ b/client/trino-jdbc/src/main/java/io/trino/jdbc/TrinoDatabaseMetaData.java @@ -20,6 +20,8 @@ import io.trino.client.ClientTypeSignatureParameter; import io.trino.client.Column; +import javax.annotation.Nullable; + import java.sql.Connection; import java.sql.DatabaseMetaData; import java.sql.ResultSet; @@ -31,6 +33,7 @@ import java.util.List; import java.util.stream.Stream; +import static com.google.common.base.Verify.verify; import static com.google.common.collect.Lists.newArrayList; import static io.trino.client.ClientTypeSignature.VARCHAR_UNBOUNDED_LENGTH; import static io.trino.jdbc.DriverInfo.DRIVER_NAME; @@ -46,10 +49,12 @@ public class TrinoDatabaseMetaData private static final String SEARCH_STRING_ESCAPE = "\\"; private final TrinoConnection connection; + private final boolean assumeLiteralNamesInMetadataCallsForNonConformingClients; - TrinoDatabaseMetaData(TrinoConnection connection) + TrinoDatabaseMetaData(TrinoConnection connection, boolean assumeLiteralNamesInMetadataCallsForNonConformingClients) { this.connection = requireNonNull(connection, "connection is null"); + this.assumeLiteralNamesInMetadataCallsForNonConformingClients = assumeLiteralNamesInMetadataCallsForNonConformingClients; } @Override @@ -897,6 +902,8 @@ public boolean dataDefinitionIgnoredInTransactions() public ResultSet getProcedures(String catalog, String schemaPattern, String procedureNamePattern) throws SQLException { + schemaPattern = escapeIfNecessary(schemaPattern); + procedureNamePattern = escapeIfNecessary(procedureNamePattern); return selectEmpty("" + "SELECT PROCEDURE_CAT, PROCEDURE_SCHEM, PROCEDURE_NAME,\n " + " null, null, null, REMARKS, PROCEDURE_TYPE, SPECIFIC_NAME\n" + @@ -908,6 +915,9 @@ public ResultSet getProcedures(String catalog, String schemaPattern, String proc public ResultSet getProcedureColumns(String catalog, String schemaPattern, String procedureNamePattern, String columnNamePattern) throws SQLException { + schemaPattern = escapeIfNecessary(schemaPattern); + procedureNamePattern = escapeIfNecessary(procedureNamePattern); + columnNamePattern = escapeIfNecessary(columnNamePattern); return selectEmpty("" + "SELECT PROCEDURE_CAT, PROCEDURE_SCHEM, PROCEDURE_NAME, " + " COLUMN_NAME, COLUMN_TYPE, DATA_TYPE, TYPE_NAME,\n" + @@ -922,6 +932,8 @@ public ResultSet getProcedureColumns(String catalog, String schemaPattern, Strin public ResultSet getTables(String catalog, String schemaPattern, String tableNamePattern, String[] types) throws SQLException { + schemaPattern = escapeIfNecessary(schemaPattern); + tableNamePattern = escapeIfNecessary(tableNamePattern); StringBuilder query = new StringBuilder("" + "SELECT TABLE_CAT, TABLE_SCHEM, TABLE_NAME, TABLE_TYPE, REMARKS,\n" + " TYPE_CAT, TYPE_SCHEM, TYPE_NAME, " + @@ -974,6 +986,9 @@ public ResultSet getTableTypes() public ResultSet getColumns(String catalog, String schemaPattern, String tableNamePattern, String columnNamePattern) throws SQLException { + schemaPattern = escapeIfNecessary(schemaPattern); + tableNamePattern = escapeIfNecessary(tableNamePattern); + columnNamePattern = escapeIfNecessary(columnNamePattern); StringBuilder query = new StringBuilder("" + "SELECT TABLE_CAT, TABLE_SCHEM, TABLE_NAME, COLUMN_NAME, DATA_TYPE,\n" + " TYPE_NAME, COLUMN_SIZE, BUFFER_LENGTH, DECIMAL_DIGITS, NUM_PREC_RADIX,\n" + @@ -999,6 +1014,7 @@ public ResultSet getColumns(String catalog, String schemaPattern, String tableNa public ResultSet getColumnPrivileges(String catalog, String schema, String table, String columnNamePattern) throws SQLException { + columnNamePattern = escapeIfNecessary(columnNamePattern); throw new SQLFeatureNotSupportedException("privileges not supported"); } @@ -1006,6 +1022,8 @@ public ResultSet getColumnPrivileges(String catalog, String schema, String table public ResultSet getTablePrivileges(String catalog, String schemaPattern, String tableNamePattern) throws SQLException { + schemaPattern = escapeIfNecessary(schemaPattern); + tableNamePattern = escapeIfNecessary(tableNamePattern); throw new SQLFeatureNotSupportedException("privileges not supported"); } @@ -1168,6 +1186,8 @@ public boolean supportsBatchUpdates() public ResultSet getUDTs(String catalog, String schemaPattern, String typeNamePattern, int[] types) throws SQLException { + schemaPattern = escapeIfNecessary(schemaPattern); + typeNamePattern = escapeIfNecessary(typeNamePattern); return selectEmpty("" + "SELECT TYPE_CAT, TYPE_SCHEM, TYPE_NAME,\n" + " CLASS_NAME, DATA_TYPE, REMARKS, BASE_TYPE\n" + @@ -1214,6 +1234,8 @@ public boolean supportsGetGeneratedKeys() public ResultSet getSuperTypes(String catalog, String schemaPattern, String typeNamePattern) throws SQLException { + schemaPattern = escapeIfNecessary(schemaPattern); + typeNamePattern = escapeIfNecessary(typeNamePattern); return selectEmpty("" + "SELECT TYPE_CAT, TYPE_SCHEM, TYPE_NAME,\n" + " SUPERTYPE_CAT, SUPERTYPE_SCHEM, SUPERTYPE_NAME\n" + @@ -1225,6 +1247,8 @@ public ResultSet getSuperTypes(String catalog, String schemaPattern, String type public ResultSet getSuperTables(String catalog, String schemaPattern, String tableNamePattern) throws SQLException { + schemaPattern = escapeIfNecessary(schemaPattern); + tableNamePattern = escapeIfNecessary(tableNamePattern); return selectEmpty("" + "SELECT TABLE_CAT, TABLE_SCHEM, TABLE_NAME, SUPERTABLE_NAME\n" + "FROM system.jdbc.super_tables\n" + @@ -1235,6 +1259,9 @@ public ResultSet getSuperTables(String catalog, String schemaPattern, String tab public ResultSet getAttributes(String catalog, String schemaPattern, String typeNamePattern, String attributeNamePattern) throws SQLException { + schemaPattern = escapeIfNecessary(schemaPattern); + typeNamePattern = escapeIfNecessary(typeNamePattern); + attributeNamePattern = escapeIfNecessary(attributeNamePattern); return selectEmpty("" + "SELECT TYPE_CAT, TYPE_SCHEM, TYPE_NAME, ATTR_NAME, DATA_TYPE,\n" + " ATTR_TYPE_NAME, ATTR_SIZE, DECIMAL_DIGITS, NUM_PREC_RADIX, NULLABLE,\n" + @@ -1332,6 +1359,7 @@ public RowIdLifetime getRowIdLifetime() public ResultSet getSchemas(String catalog, String schemaPattern) throws SQLException { + schemaPattern = escapeIfNecessary(schemaPattern); StringBuilder query = new StringBuilder("" + "SELECT TABLE_SCHEM, TABLE_CATALOG\n" + "FROM system.jdbc.schemas"); @@ -1391,6 +1419,8 @@ public ResultSet getClientInfoProperties() public ResultSet getFunctions(String catalog, String schemaPattern, String functionNamePattern) throws SQLException { + schemaPattern = escapeIfNecessary(schemaPattern); + functionNamePattern = escapeIfNecessary(functionNamePattern); // TODO: implement this throw new NotImplementedException("DatabaseMetaData", "getFunctions"); } @@ -1399,6 +1429,9 @@ public ResultSet getFunctions(String catalog, String schemaPattern, String funct public ResultSet getFunctionColumns(String catalog, String schemaPattern, String functionNamePattern, String columnNamePattern) throws SQLException { + schemaPattern = escapeIfNecessary(schemaPattern); + functionNamePattern = escapeIfNecessary(functionNamePattern); + columnNamePattern = escapeIfNecessary(columnNamePattern); // TODO: implement this throw new NotImplementedException("DatabaseMetaData", "getFunctionColumns"); } @@ -1407,6 +1440,9 @@ public ResultSet getFunctionColumns(String catalog, String schemaPattern, String public ResultSet getPseudoColumns(String catalog, String schemaPattern, String tableNamePattern, String columnNamePattern) throws SQLException { + schemaPattern = escapeIfNecessary(schemaPattern); + tableNamePattern = escapeIfNecessary(tableNamePattern); + columnNamePattern = escapeIfNecessary(columnNamePattern); return selectEmpty("" + "SELECT TABLE_CAT, TABLE_SCHEM, TABLE_NAME, COLUMN_NAME, DATA_TYPE,\n" + " COLUMN_SIZE, DECIMAL_DIGITS, NUM_PREC_RADIX, COLUMN_USAGE, REMARKS,\n" + @@ -1485,6 +1521,23 @@ private static void optionalStringInFilter(List filters, String columnNa filters.add(filter.toString()); } + @Nullable + private String escapeIfNecessary(@Nullable String namePattern) + { + return escapeIfNecessary(assumeLiteralNamesInMetadataCallsForNonConformingClients, namePattern); + } + + @Nullable + static String escapeIfNecessary(boolean assumeLiteralNamesInMetadataCallsForNonConformingClients, @Nullable String namePattern) + { + if (namePattern == null || !assumeLiteralNamesInMetadataCallsForNonConformingClients) { + return namePattern; + } + //noinspection ConstantConditions + verify(SEARCH_STRING_ESCAPE.equals("\\")); + return namePattern.replaceAll("[_%\\\\]", "\\\\$0"); + } + private static void optionalStringLikeFilter(List filters, String columnName, String value) { if (value != null) { diff --git a/client/trino-jdbc/src/main/java/io/trino/jdbc/TrinoDriverUri.java b/client/trino-jdbc/src/main/java/io/trino/jdbc/TrinoDriverUri.java index 9a2af8cb55d0..b4c8e6121d64 100644 --- a/client/trino-jdbc/src/main/java/io/trino/jdbc/TrinoDriverUri.java +++ b/client/trino-jdbc/src/main/java/io/trino/jdbc/TrinoDriverUri.java @@ -53,11 +53,13 @@ import static io.trino.client.OkHttpUtil.tokenAuth; import static io.trino.jdbc.ConnectionProperties.ACCESS_TOKEN; import static io.trino.jdbc.ConnectionProperties.APPLICATION_NAME_PREFIX; +import static io.trino.jdbc.ConnectionProperties.ASSUME_LITERAL_NAMES_IN_METADATA_CALLS_FOR_NON_CONFORMING_CLIENTS; import static io.trino.jdbc.ConnectionProperties.CLIENT_INFO; import static io.trino.jdbc.ConnectionProperties.CLIENT_TAGS; import static io.trino.jdbc.ConnectionProperties.DISABLE_COMPRESSION; import static io.trino.jdbc.ConnectionProperties.EXTERNAL_AUTHENTICATION; import static io.trino.jdbc.ConnectionProperties.EXTERNAL_AUTHENTICATION_TIMEOUT; +import static io.trino.jdbc.ConnectionProperties.EXTERNAL_AUTHENTICATION_TOKEN_CACHE; import static io.trino.jdbc.ConnectionProperties.EXTRA_CREDENTIALS; import static io.trino.jdbc.ConnectionProperties.HTTP_PROXY; import static io.trino.jdbc.ConnectionProperties.KERBEROS_CONFIG_PATH; @@ -236,6 +238,12 @@ public boolean isCompressionDisabled() return DISABLE_COMPRESSION.getValue(properties).orElse(false); } + public boolean isAssumeLiteralNamesInMetadataCallsForNonConformingClients() + throws SQLException + { + return ASSUME_LITERAL_NAMES_IN_METADATA_CALLS_FOR_NON_CONFORMING_CLIENTS.getValue(properties).orElse(false); + } + public void setupClient(OkHttpClient.Builder builder) throws SQLException { @@ -310,7 +318,9 @@ public void setupClient(OkHttpClient.Builder builder) .map(value -> Duration.ofMillis(value.toMillis())) .orElse(Duration.ofMinutes(2)); - ExternalAuthenticator authenticator = new ExternalAuthenticator(REDIRECT_HANDLER.get(), poller, timeout); + KnownTokenCache knownTokenCache = EXTERNAL_AUTHENTICATION_TOKEN_CACHE.getValue(properties).get(); + + ExternalAuthenticator authenticator = new ExternalAuthenticator(REDIRECT_HANDLER.get(), poller, knownTokenCache.create(), timeout); builder.authenticator(authenticator); builder.addInterceptor(authenticator); diff --git a/client/trino-jdbc/src/test/java/io/trino/jdbc/TestQueryExecutor.java b/client/trino-jdbc/src/test/java/io/trino/jdbc/TestQueryExecutor.java index d800eb6aeb16..d3154921f8cf 100644 --- a/client/trino-jdbc/src/test/java/io/trino/jdbc/TestQueryExecutor.java +++ b/client/trino-jdbc/src/test/java/io/trino/jdbc/TestQueryExecutor.java @@ -29,6 +29,7 @@ import static com.google.common.net.HttpHeaders.CONTENT_TYPE; import static io.airlift.json.JsonCodec.jsonCodec; import static io.trino.client.NodeVersion.UNKNOWN; +import static java.util.concurrent.TimeUnit.MINUTES; import static org.testng.Assert.assertEquals; @Test(singleThreaded = true) @@ -57,7 +58,7 @@ public void teardown() public void testGetServerInfo() throws Exception { - ServerInfo expected = new ServerInfo(UNKNOWN, "test", true, false, Optional.of(Duration.valueOf("2m"))); + ServerInfo expected = new ServerInfo(UNKNOWN, "test", true, false, Optional.of(new Duration(2, MINUTES))); server.enqueue(new MockResponse() .addHeader(CONTENT_TYPE, "application/json") @@ -67,7 +68,7 @@ public void testGetServerInfo() ServerInfo actual = executor.getServerInfo(server.url("/v1/info").uri()); assertEquals(actual.getEnvironment(), "test"); - assertEquals(actual.getUptime(), Optional.of(Duration.valueOf("2m"))); + assertEquals(actual.getUptime(), Optional.of(new Duration(2, MINUTES))); assertEquals(server.getRequestCount(), 1); assertEquals(server.takeRequest().getPath(), "/v1/info"); diff --git a/client/trino-jdbc/src/test/java/io/trino/jdbc/TestTrinoDatabaseMetaData.java b/client/trino-jdbc/src/test/java/io/trino/jdbc/TestTrinoDatabaseMetaData.java index 26efb600646b..924347f50f57 100644 --- a/client/trino-jdbc/src/test/java/io/trino/jdbc/TestTrinoDatabaseMetaData.java +++ b/client/trino-jdbc/src/test/java/io/trino/jdbc/TestTrinoDatabaseMetaData.java @@ -987,7 +987,6 @@ public void testGetSuperTypes() } @Test - @SuppressWarnings("resource") public void testGetSchemasMetadataCalls() throws Exception { @@ -995,6 +994,7 @@ public void testGetSchemasMetadataCalls() // No filter assertMetadataCalls( + connection, readMetaData( databaseMetaData -> databaseMetaData.getSchemas(null, null), list("TABLE_CATALOG", "TABLE_SCHEM")), @@ -1003,6 +1003,7 @@ public void testGetSchemasMetadataCalls() // Equality predicate on catalog name assertMetadataCalls( + connection, readMetaData( databaseMetaData -> databaseMetaData.getSchemas(COUNTING_CATALOG, null), list("TABLE_CATALOG", "TABLE_SCHEM")), @@ -1015,6 +1016,7 @@ public void testGetSchemasMetadataCalls() // Equality predicate on schema name assertMetadataCalls( + connection, readMetaData( databaseMetaData -> databaseMetaData.getSchemas(COUNTING_CATALOG, "test\\_schema%"), list("TABLE_CATALOG", "TABLE_SCHEM")), @@ -1026,6 +1028,7 @@ public void testGetSchemasMetadataCalls() // LIKE predicate on schema name assertMetadataCalls( + connection, readMetaData( databaseMetaData -> databaseMetaData.getSchemas(COUNTING_CATALOG, "test_sch_ma1"), list("TABLE_CATALOG", "TABLE_SCHEM")), @@ -1035,6 +1038,7 @@ public void testGetSchemasMetadataCalls() // Empty schema name assertMetadataCalls( + connection, readMetaData( databaseMetaData -> databaseMetaData.getSchemas(COUNTING_CATALOG, ""), list("TABLE_CATALOG", "TABLE_SCHEM")), @@ -1044,6 +1048,7 @@ public void testGetSchemasMetadataCalls() // catalog does not exist assertMetadataCalls( + connection, readMetaData( databaseMetaData -> databaseMetaData.getSchemas("wrong", null), list("TABLE_CATALOG", "TABLE_SCHEM")), @@ -1052,7 +1057,6 @@ public void testGetSchemasMetadataCalls() } @Test - @SuppressWarnings("resource") public void testGetTablesMetadataCalls() throws Exception { @@ -1060,6 +1064,7 @@ public void testGetTablesMetadataCalls() // No filter assertMetadataCalls( + connection, readMetaData( databaseMetaData -> databaseMetaData.getTables(null, null, null, null), list("TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME", "TABLE_TYPE")), @@ -1069,6 +1074,7 @@ public void testGetTablesMetadataCalls() // Equality predicate on catalog name assertMetadataCalls( + connection, readMetaData( databaseMetaData -> databaseMetaData.getTables(COUNTING_CATALOG, null, null, null), list("TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME", "TABLE_TYPE")), @@ -1078,6 +1084,7 @@ public void testGetTablesMetadataCalls() // Equality predicate on schema name assertMetadataCalls( + connection, readMetaData( databaseMetaData -> databaseMetaData.getTables(COUNTING_CATALOG, "test\\_schema1", null, null), list("TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME", "TABLE_TYPE")), @@ -1090,6 +1097,7 @@ public void testGetTablesMetadataCalls() // LIKE predicate on schema name assertMetadataCalls( + connection, readMetaData( databaseMetaData -> databaseMetaData.getTables(COUNTING_CATALOG, "test_sch_ma1", null, null), list("TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME", "TABLE_TYPE")), @@ -1103,6 +1111,7 @@ public void testGetTablesMetadataCalls() // Equality predicate on table name assertMetadataCalls( + connection, readMetaData( databaseMetaData -> databaseMetaData.getTables(COUNTING_CATALOG, null, "test\\_table1", null), list("TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME", "TABLE_TYPE")), @@ -1115,6 +1124,7 @@ public void testGetTablesMetadataCalls() // LIKE predicate on table name assertMetadataCalls( + connection, readMetaData( databaseMetaData -> databaseMetaData.getTables(COUNTING_CATALOG, null, "test_t_ble1", null), list("TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME", "TABLE_TYPE")), @@ -1127,14 +1137,17 @@ public void testGetTablesMetadataCalls() // Equality predicate on schema name and table name assertMetadataCalls( + connection, readMetaData( databaseMetaData -> databaseMetaData.getTables(COUNTING_CATALOG, "test\\_schema1", "test\\_table1", null), list("TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME", "TABLE_TYPE")), list(list(COUNTING_CATALOG, "test_schema1", "test_table1", "TABLE")), - new MetadataCallsCount()); + new MetadataCallsCount() + .withGetTableHandleCount(1)); // LIKE predicate on schema name and table name assertMetadataCalls( + connection, readMetaData( databaseMetaData -> databaseMetaData.getTables(COUNTING_CATALOG, "test_schema1", "test_table1", null), list("TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME", "TABLE_TYPE")), @@ -1145,6 +1158,7 @@ public void testGetTablesMetadataCalls() // catalog does not exist assertMetadataCalls( + connection, readMetaData( databaseMetaData -> databaseMetaData.getTables("wrong", null, null, null), list("TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME", "TABLE_TYPE")), @@ -1153,6 +1167,7 @@ public void testGetTablesMetadataCalls() // empty schema name assertMetadataCalls( + connection, readMetaData( databaseMetaData -> databaseMetaData.getTables(COUNTING_CATALOG, "", null, null), list("TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME", "TABLE_TYPE")), @@ -1163,6 +1178,7 @@ public void testGetTablesMetadataCalls() // empty table name assertMetadataCalls( + connection, readMetaData( databaseMetaData -> databaseMetaData.getTables(COUNTING_CATALOG, null, "", null), list("TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME", "TABLE_TYPE")), @@ -1173,6 +1189,7 @@ public void testGetTablesMetadataCalls() // no table types selected assertMetadataCalls( + connection, readMetaData( databaseMetaData -> databaseMetaData.getTables(COUNTING_CATALOG, null, null, new String[0]), list("TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME", "TABLE_TYPE")), @@ -1181,7 +1198,6 @@ public void testGetTablesMetadataCalls() } @Test - @SuppressWarnings("resource") public void testGetColumnsMetadataCalls() throws Exception { @@ -1189,6 +1205,7 @@ public void testGetColumnsMetadataCalls() // No filter assertMetadataCalls( + connection, readMetaData( databaseMetaData -> databaseMetaData.getColumns(null, null, null, null), list("TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME", "COLUMN_NAME", "TYPE_NAME")), @@ -1199,6 +1216,7 @@ public void testGetColumnsMetadataCalls() // Equality predicate on catalog name assertMetadataCalls( + connection, readMetaData( databaseMetaData -> databaseMetaData.getColumns(COUNTING_CATALOG, null, null, null), list("TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME", "COLUMN_NAME", "TYPE_NAME")), @@ -1209,6 +1227,7 @@ public void testGetColumnsMetadataCalls() // Equality predicate on catalog name, schema name and table name assertMetadataCalls( + connection, readMetaData( databaseMetaData -> databaseMetaData.getColumns(COUNTING_CATALOG, "test\\_schema1", "test\\_table1", null), list("TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME", "COLUMN_NAME", "TYPE_NAME")), @@ -1221,6 +1240,7 @@ public void testGetColumnsMetadataCalls() // Equality predicate on catalog name, schema name, table name and column name assertMetadataCalls( + connection, readMetaData( databaseMetaData -> databaseMetaData.getColumns(COUNTING_CATALOG, "test\\_schema1", "test\\_table1", "column\\_17"), list("TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME", "COLUMN_NAME", "TYPE_NAME")), @@ -1231,6 +1251,7 @@ public void testGetColumnsMetadataCalls() // Equality predicate on catalog name, LIKE predicate on schema name, table name and column name assertMetadataCalls( + connection, readMetaData( databaseMetaData -> databaseMetaData.getColumns(COUNTING_CATALOG, "test_schema1", "test_table1", "column_17"), list("TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME", "COLUMN_NAME", "TYPE_NAME")), @@ -1242,6 +1263,7 @@ public void testGetColumnsMetadataCalls() // LIKE predicate on schema name and table name, but no predicate on catalog name assertMetadataCalls( + connection, readMetaData( databaseMetaData -> databaseMetaData.getColumns(null, "test_schema1", "test_table1", null), list("TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME", "COLUMN_NAME", "TYPE_NAME")), @@ -1255,6 +1277,7 @@ public void testGetColumnsMetadataCalls() // LIKE predicate on schema name, but no predicate on catalog name and table name assertMetadataCalls( + connection, readMetaData( databaseMetaData -> databaseMetaData.getColumns(null, "test_schema1", null, null), list("TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME", "COLUMN_NAME", "TYPE_NAME")), @@ -1270,6 +1293,7 @@ public void testGetColumnsMetadataCalls() // LIKE predicate on table name, but no predicate on catalog name and schema name assertMetadataCalls( + connection, readMetaData( databaseMetaData -> databaseMetaData.getColumns(null, null, "test_table1", null), list("TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME", "COLUMN_NAME", "TYPE_NAME")), @@ -1281,10 +1305,12 @@ public void testGetColumnsMetadataCalls() new MetadataCallsCount() .withListSchemasCount(3) .withListTablesCount(8) + .withGetTableHandleCount(2) .withGetColumnsCount(2)); // Equality predicate on schema name and table name, but no predicate on catalog name assertMetadataCalls( + connection, readMetaData( databaseMetaData -> databaseMetaData.getColumns(null, "test\\_schema1", "test\\_table1", null), list("TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME", "COLUMN_NAME", "TYPE_NAME")), @@ -1297,6 +1323,7 @@ public void testGetColumnsMetadataCalls() // catalog does not exist assertMetadataCalls( + connection, readMetaData( databaseMetaData -> databaseMetaData.getColumns("wrong", null, null, null), list("TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME", "COLUMN_NAME", "TYPE_NAME")), @@ -1305,6 +1332,7 @@ public void testGetColumnsMetadataCalls() // schema does not exist assertMetadataCalls( + connection, readMetaData( databaseMetaData -> databaseMetaData.getColumns(COUNTING_CATALOG, "wrong\\_schema1", "test\\_table1", null), list("TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME", "COLUMN_NAME", "TYPE_NAME")), @@ -1314,6 +1342,7 @@ public void testGetColumnsMetadataCalls() // schema does not exist assertMetadataCalls( + connection, readMetaData( databaseMetaData -> databaseMetaData.getColumns(COUNTING_CATALOG, "wrong_schema1", "test_table1", null), list("TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME", "COLUMN_NAME", "TYPE_NAME")), @@ -1325,6 +1354,7 @@ public void testGetColumnsMetadataCalls() // empty schema name assertMetadataCalls( + connection, readMetaData( databaseMetaData -> databaseMetaData.getColumns(COUNTING_CATALOG, "", null, null), list("TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME", "COLUMN_NAME", "TYPE_NAME")), @@ -1336,6 +1366,7 @@ public void testGetColumnsMetadataCalls() // empty table name assertMetadataCalls( + connection, readMetaData( databaseMetaData -> databaseMetaData.getColumns(COUNTING_CATALOG, null, "", null), list("TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME", "COLUMN_NAME", "TYPE_NAME")), @@ -1347,6 +1378,7 @@ public void testGetColumnsMetadataCalls() // empty column name assertMetadataCalls( + connection, readMetaData( databaseMetaData -> databaseMetaData.getColumns(COUNTING_CATALOG, null, null, ""), list("TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME", "COLUMN_NAME", "TYPE_NAME")), @@ -1357,6 +1389,102 @@ public void testGetColumnsMetadataCalls() .withGetColumnsCount(3000)); } + @Test + public void testAssumeLiteralMetadataCalls() + throws Exception + { + try (Connection connection = DriverManager.getConnection( + format("jdbc:trino://%s?assumeLiteralNamesInMetadataCallsForNonConformingClients=true", server.getAddress()), + "admin", + null)) { + // getTables's schema name pattern treated as literal + assertMetadataCalls( + connection, + readMetaData( + databaseMetaData -> databaseMetaData.getTables(COUNTING_CATALOG, "test_schema1", null, null), + list("TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME", "TABLE_TYPE")), + countingMockConnector.getAllTables() + .filter(schemaTableName -> schemaTableName.getSchemaName().equals("test_schema1")) + .map(schemaTableName -> list(COUNTING_CATALOG, schemaTableName.getSchemaName(), schemaTableName.getTableName(), "TABLE")) + .collect(toImmutableList()), + new MetadataCallsCount() + .withListSchemasCount(0) + .withListTablesCount(1)); + + // getTables's schema and table name patterns treated as literals + assertMetadataCalls( + connection, + readMetaData( + databaseMetaData -> databaseMetaData.getTables(COUNTING_CATALOG, "test_schema1", "test_table1", null), + list("TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME", "TABLE_TYPE")), + list(list(COUNTING_CATALOG, "test_schema1", "test_table1", "TABLE")), + new MetadataCallsCount() + .withGetTableHandleCount(1)); + + // no matches in getTables call as table name pattern treated as literal + assertMetadataCalls( + connection, + readMetaData( + databaseMetaData -> databaseMetaData.getTables(COUNTING_CATALOG, "test_schema_", null, null), + list("TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME", "TABLE_TYPE")), + list(), + new MetadataCallsCount() + .withListTablesCount(1)); + + // getColumns's schema and table name patterns treated as literals + assertMetadataCalls( + connection, + readMetaData( + databaseMetaData -> databaseMetaData.getColumns(COUNTING_CATALOG, "test_schema1", "test_table1", null), + list("TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME", "COLUMN_NAME", "TYPE_NAME")), + IntStream.range(0, 100) + .mapToObj(i -> list(COUNTING_CATALOG, "test_schema1", "test_table1", "column_" + i, "varchar")) + .collect(toImmutableList()), + new MetadataCallsCount() + .withListTablesCount(1) + .withGetColumnsCount(1)); + + // getColumns's schema, table and column name patterns treated as literals + assertMetadataCalls( + connection, + readMetaData( + databaseMetaData -> databaseMetaData.getColumns(COUNTING_CATALOG, "test_schema1", "test_table1", "column_17"), + list("TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME", "COLUMN_NAME", "TYPE_NAME")), + list(list(COUNTING_CATALOG, "test_schema1", "test_table1", "column_17", "varchar")), + new MetadataCallsCount() + .withListTablesCount(1) + .withGetColumnsCount(1)); + + // no matches in getColumns call as table name pattern treated as literal + assertMetadataCalls( + connection, + readMetaData( + databaseMetaData -> databaseMetaData.getColumns(COUNTING_CATALOG, "test_schema1", "test_table_", null), + list("TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME", "COLUMN_NAME", "TYPE_NAME")), + list(), + new MetadataCallsCount() + .withListTablesCount(1)); + } + } + + @Test + public void testEscapeIfNecessary() + { + assertEquals(TrinoDatabaseMetaData.escapeIfNecessary(false, null), null); + assertEquals(TrinoDatabaseMetaData.escapeIfNecessary(false, "a"), "a"); + assertEquals(TrinoDatabaseMetaData.escapeIfNecessary(false, "abc_def"), "abc_def"); + assertEquals(TrinoDatabaseMetaData.escapeIfNecessary(false, "abc__de_f"), "abc__de_f"); + assertEquals(TrinoDatabaseMetaData.escapeIfNecessary(false, "abc%def"), "abc%def"); + assertEquals(TrinoDatabaseMetaData.escapeIfNecessary(false, "abc\\_def"), "abc\\_def"); + + assertEquals(TrinoDatabaseMetaData.escapeIfNecessary(true, null), null); + assertEquals(TrinoDatabaseMetaData.escapeIfNecessary(true, "a"), "a"); + assertEquals(TrinoDatabaseMetaData.escapeIfNecessary(true, "abc_def"), "abc\\_def"); + assertEquals(TrinoDatabaseMetaData.escapeIfNecessary(true, "abc__de_f"), "abc\\_\\_de\\_f"); + assertEquals(TrinoDatabaseMetaData.escapeIfNecessary(true, "abc%def"), "abc\\%def"); + assertEquals(TrinoDatabaseMetaData.escapeIfNecessary(true, "abc\\_def"), "abc\\\\\\_def"); + } + private static void assertColumnSpec(ResultSet rs, int dataType, Long precision, Long numPrecRadix, String typeName) throws SQLException { @@ -1396,19 +1524,23 @@ private Set captureQueries(Callable action) .collect(toImmutableSet()); } - private void assertMetadataCalls(MetaDataCallback>> callback, MetadataCallsCount expectedMetadataCallsCount) - throws Exception + private void assertMetadataCalls(Connection connection, MetaDataCallback>> callback, MetadataCallsCount expectedMetadataCallsCount) { assertMetadataCalls( + connection, callback, actual -> {}, expectedMetadataCallsCount); } - private void assertMetadataCalls(MetaDataCallback>> callback, Collection> expected, MetadataCallsCount expectedMetadataCallsCount) - throws Exception + private void assertMetadataCalls( + Connection connection, + MetaDataCallback>> callback, + Collection> expected, + MetadataCallsCount expectedMetadataCallsCount) { assertMetadataCalls( + connection, callback, actual -> assertThat(ImmutableMultiset.copyOf(requireNonNull(actual, "actual is null"))) .isEqualTo(ImmutableMultiset.copyOf(requireNonNull(expected, "expected is null"))), @@ -1416,23 +1548,20 @@ private void assertMetadataCalls(MetaDataCallback>> callback, Consumer>> resultsVerification, MetadataCallsCount expectedMetadataCallsCount) - throws Exception { - MetadataCallsCount actualMetadataCallsCount; - try (Connection connection = createConnection()) { - actualMetadataCallsCount = countingMockConnector.runCounting(() -> { - try { - Collection> actual = callback.apply(connection.getMetaData()); - resultsVerification.accept(actual); - } - catch (SQLException e) { - throw new RuntimeException(e); - } - }); - } + MetadataCallsCount actualMetadataCallsCount = countingMockConnector.runCounting(() -> { + try { + Collection> actual = callback.apply(connection.getMetaData()); + resultsVerification.accept(actual); + } + catch (SQLException e) { + throw new RuntimeException(e); + } + }); assertEquals(actualMetadataCallsCount, expectedMetadataCallsCount); } diff --git a/client/trino-jdbc/src/test/java/io/trino/jdbc/TestTrinoDriverImpersonateUser.java b/client/trino-jdbc/src/test/java/io/trino/jdbc/TestTrinoDriverImpersonateUser.java index 30876bc445cd..1a23e854a4e3 100644 --- a/client/trino-jdbc/src/test/java/io/trino/jdbc/TestTrinoDriverImpersonateUser.java +++ b/client/trino-jdbc/src/test/java/io/trino/jdbc/TestTrinoDriverImpersonateUser.java @@ -23,6 +23,9 @@ import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; import java.security.Principal; import java.sql.Connection; import java.sql.DriverManager; @@ -47,9 +50,13 @@ public class TestTrinoDriverImpersonateUser @BeforeClass public void setup() + throws IOException { + Path passwordConfigDummy = Files.createTempFile("passwordConfigDummy", null); + passwordConfigDummy.toFile().deleteOnExit(); server = TestingTrinoServer.builder() .setProperties(ImmutableMap.builder() + .put("password-authenticator.config-files", passwordConfigDummy.toString()) .put("http-server.authentication.type", "password") .put("http-server.https.enabled", "true") .put("http-server.https.keystore.path", getResource("localhost.keystore").getPath()) @@ -57,7 +64,7 @@ public void setup() .build()) .build(); - server.getInstance(Key.get(PasswordAuthenticatorManager.class)).setAuthenticator(TestTrinoDriverImpersonateUser::authenticate); + server.getInstance(Key.get(PasswordAuthenticatorManager.class)).setAuthenticators(TestTrinoDriverImpersonateUser::authenticate); } private static Principal authenticate(String user, String password) diff --git a/core/trino-main/pom.xml b/core/trino-main/pom.xml index 83baff416725..521dee25ed1b 100644 --- a/core/trino-main/pom.xml +++ b/core/trino-main/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/core/trino-main/src/main/java/io/trino/event/QueryMonitor.java b/core/trino-main/src/main/java/io/trino/event/QueryMonitor.java index d276de194c89..ae193965100e 100644 --- a/core/trino-main/src/main/java/io/trino/event/QueryMonitor.java +++ b/core/trino-main/src/main/java/io/trino/event/QueryMonitor.java @@ -44,6 +44,7 @@ import io.trino.operator.TaskStats; import io.trino.server.BasicQueryInfo; import io.trino.spi.QueryId; +import io.trino.spi.eventlistener.OutputColumnMetadata; import io.trino.spi.eventlistener.QueryCompletedEvent; import io.trino.spi.eventlistener.QueryContext; import io.trino.spi.eventlistener.QueryCreatedEvent; @@ -56,6 +57,7 @@ import io.trino.spi.eventlistener.StageCpuDistribution; import io.trino.spi.resourcegroups.QueryType; import io.trino.spi.resourcegroups.ResourceGroupId; +import io.trino.sql.analyzer.Analysis; import io.trino.sql.planner.PlanFragment; import io.trino.sql.planner.plan.PlanFragmentId; import io.trino.sql.planner.plan.PlanNode; @@ -76,6 +78,8 @@ import java.util.OptionalLong; import java.util.stream.Collectors; +import static com.google.common.collect.ImmutableList.toImmutableList; +import static com.google.common.collect.ImmutableSet.toImmutableSet; import static io.trino.execution.QueryState.QUEUED; import static io.trino.execution.StageInfo.getAllStages; import static io.trino.sql.planner.planprinter.PlanPrinter.textDistributedPlan; @@ -362,11 +366,21 @@ private static QueryIOMetadata getQueryIOMetadata(QueryInfo queryInfo) .map(TableFinishInfo.class::cast) .findFirst(); + Optional> outputColumnsMetadata = queryInfo.getOutput().get().getColumns() + .map(columns -> columns.stream() + .map(column -> new OutputColumnMetadata( + column.getColumn().getName(), + column.getSourceColumns().stream() + .map(Analysis.SourceColumn::getColumnDetail) + .collect(toImmutableSet()))) + .collect(toImmutableList())); + output = Optional.of( new QueryOutputMetadata( queryInfo.getOutput().get().getCatalogName(), queryInfo.getOutput().get().getSchema(), queryInfo.getOutput().get().getTable(), + outputColumnsMetadata, tableFinishInfo.map(TableFinishInfo::getConnectorOutputMetadata), tableFinishInfo.map(TableFinishInfo::isJsonLengthLimitExceeded))); } diff --git a/core/trino-main/src/main/java/io/trino/execution/CreateMaterializedViewTask.java b/core/trino-main/src/main/java/io/trino/execution/CreateMaterializedViewTask.java index fa08dde438a2..a26bb61e4885 100644 --- a/core/trino-main/src/main/java/io/trino/execution/CreateMaterializedViewTask.java +++ b/core/trino-main/src/main/java/io/trino/execution/CreateMaterializedViewTask.java @@ -99,7 +99,7 @@ public ListenableFuture execute( .map(field -> new ConnectorMaterializedViewDefinition.Column(field.getName().get(), field.getType().getTypeId())) .collect(toImmutableList()); - Optional owner = Optional.of(session.getUser()); + String owner = session.getUser(); CatalogName catalogName = metadata.getCatalogHandle(session, name.getCatalogName()) .orElseThrow(() -> new TrinoException(NOT_FOUND, "Catalog does not exist: " + name.getCatalogName())); diff --git a/core/trino-main/src/main/java/io/trino/execution/CreateTableTask.java b/core/trino-main/src/main/java/io/trino/execution/CreateTableTask.java index 84ffa69d33de..0ea34af2dcf4 100644 --- a/core/trino-main/src/main/java/io/trino/execution/CreateTableTask.java +++ b/core/trino-main/src/main/java/io/trino/execution/CreateTableTask.java @@ -16,6 +16,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; import com.google.common.util.concurrent.ListenableFuture; import io.trino.Session; import io.trino.connector.CatalogName; @@ -31,6 +32,8 @@ import io.trino.spi.security.AccessDeniedException; import io.trino.spi.type.Type; import io.trino.spi.type.TypeNotFoundException; +import io.trino.sql.analyzer.Output; +import io.trino.sql.analyzer.OutputColumn; import io.trino.sql.tree.ColumnDefinition; import io.trino.sql.tree.CreateTable; import io.trino.sql.tree.Expression; @@ -47,8 +50,10 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.function.Consumer; import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.collect.ImmutableList.toImmutableList; import static com.google.common.collect.ImmutableSet.toImmutableSet; import static com.google.common.util.concurrent.Futures.immediateFuture; import static io.trino.metadata.MetadataUtil.createQualifiedObjectName; @@ -96,11 +101,11 @@ public ListenableFuture execute( List parameters, WarningCollector warningCollector) { - return internalExecute(statement, metadata, accessControl, stateMachine.getSession(), parameters); + return internalExecute(statement, metadata, accessControl, stateMachine.getSession(), parameters, output -> stateMachine.setOutput(Optional.of(output))); } @VisibleForTesting - ListenableFuture internalExecute(CreateTable statement, Metadata metadata, AccessControl accessControl, Session session, List parameters) + ListenableFuture internalExecute(CreateTable statement, Metadata metadata, AccessControl accessControl, Session session, List parameters, Consumer outputConsumer) { checkArgument(!statement.getElements().isEmpty(), "no columns for table"); @@ -240,6 +245,13 @@ else if (element instanceof LikeClause) { throw e; } } + outputConsumer.accept(new Output( + tableName.getCatalogName(), + tableName.getSchemaName(), + tableName.getObjectName(), + Optional.of(tableMetadata.getColumns().stream() + .map(column -> new OutputColumn(new Column(column.getName(), column.getType().toString()), ImmutableSet.of())) + .collect(toImmutableList())))); return immediateFuture(null); } diff --git a/core/trino-main/src/main/java/io/trino/metadata/FunctionRegistry.java b/core/trino-main/src/main/java/io/trino/metadata/FunctionRegistry.java index 4f2512edee59..79b59e7e5d69 100644 --- a/core/trino-main/src/main/java/io/trino/metadata/FunctionRegistry.java +++ b/core/trino-main/src/main/java/io/trino/metadata/FunctionRegistry.java @@ -192,7 +192,6 @@ import io.trino.operator.scalar.timestamp.TimestampToTimestampWithTimeZoneCast; import io.trino.operator.scalar.timestamp.TimestampToVarcharCast; import io.trino.operator.scalar.timestamp.ToIso8601; -import io.trino.operator.scalar.timestamp.ToUnixTime; import io.trino.operator.scalar.timestamp.VarcharToTimestampCast; import io.trino.operator.scalar.timestamp.WithTimeZone; import io.trino.operator.scalar.timestamptz.AtTimeZone; @@ -632,7 +631,6 @@ public FunctionRegistry( .scalar(VarcharToTimestampCast.class) .scalar(LocalTimestamp.class) .scalar(DateTrunc.class) - .scalar(ToUnixTime.class) .scalar(HumanReadableSeconds.class) .scalar(ToIso8601.class) .scalar(WithTimeZone.class) diff --git a/core/trino-main/src/main/java/io/trino/metadata/Metadata.java b/core/trino-main/src/main/java/io/trino/metadata/Metadata.java index f6f33ec33588..bb76cd1e54a8 100644 --- a/core/trino-main/src/main/java/io/trino/metadata/Metadata.java +++ b/core/trino-main/src/main/java/io/trino/metadata/Metadata.java @@ -114,8 +114,19 @@ public interface Metadata Optional getInfo(Session session, TableHandle handle); + /** + * Return table schema definition for the specified table handle. + * Table schema definition is a set of information + * required by semantic analyzer to analyze the query. + * @see {@link #getTableMetadata(Session, TableHandle)} + * + * @throws RuntimeException if table handle is no longer valid + */ + TableSchema getTableSchema(Session session, TableHandle tableHandle); + /** * Return the metadata for the specified table handle. + * @see {@link #getTableSchema(Session, TableHandle)} which is less expsensive. * * @throws RuntimeException if table handle is no longer valid */ diff --git a/core/trino-main/src/main/java/io/trino/metadata/MetadataManager.java b/core/trino-main/src/main/java/io/trino/metadata/MetadataManager.java index 9807e9d9a495..b65adbca7665 100644 --- a/core/trino-main/src/main/java/io/trino/metadata/MetadataManager.java +++ b/core/trino-main/src/main/java/io/trino/metadata/MetadataManager.java @@ -15,11 +15,15 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Joiner; +import com.google.common.cache.CacheBuilder; +import com.google.common.cache.CacheLoader; +import com.google.common.cache.LoadingCache; import com.google.common.collect.HashMultimap; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Multimap; +import com.google.common.util.concurrent.UncheckedExecutionException; import io.airlift.slice.Slice; import io.trino.Session; import io.trino.client.NodeVersion; @@ -69,6 +73,7 @@ import io.trino.spi.connector.ConnectorTableLayoutResult; import io.trino.spi.connector.ConnectorTableMetadata; import io.trino.spi.connector.ConnectorTableProperties; +import io.trino.spi.connector.ConnectorTableSchema; import io.trino.spi.connector.ConnectorTransactionHandle; import io.trino.spi.connector.ConnectorViewDefinition; import io.trino.spi.connector.Constraint; @@ -200,6 +205,9 @@ public final class MetadataManager private final ResolvedFunctionDecoder functionDecoder; + private final LoadingCache operatorCache; + private final LoadingCache coercionCache; + @Inject public MetadataManager( FeaturesConfig featuresConfig, @@ -247,6 +255,23 @@ public MetadataManager( verifyTypes(); functionDecoder = new ResolvedFunctionDecoder(this::getType); + + operatorCache = CacheBuilder.newBuilder() + .maximumSize(1000) + .build(CacheLoader.from(key -> { + String name = mangleOperatorName(key.getOperatorType()); + return resolveFunction(QualifiedName.of(name), fromTypes(key.getArgumentTypes())); + })); + + coercionCache = CacheBuilder.newBuilder() + .maximumSize(1000) + .build(CacheLoader.from(key -> { + String name = mangleOperatorName(key.getOperatorType()); + Type fromType = key.getFromType(); + Type toType = key.getToType(); + Signature signature = new Signature(name, toType.getTypeSignature(), ImmutableList.of(fromType.getTypeSignature())); + return resolve(functionResolver.resolveCoercion(functions.get(QualifiedName.of(name)), signature)); + })); } public static MetadataManager createTestMetadataManager() @@ -508,6 +533,16 @@ public Optional getInfo(Session session, TableHandle handle) return metadata.getInfo(handle.getConnectorHandle()); } + @Override + public TableSchema getTableSchema(Session session, TableHandle tableHandle) + { + CatalogName catalogName = tableHandle.getCatalogName(); + ConnectorMetadata metadata = getMetadata(session, catalogName); + ConnectorTableSchema tableSchema = metadata.getTableSchema(session.toConnectorSession(catalogName), tableHandle.getConnectorHandle()); + + return new TableSchema(catalogName, tableSchema); + } + @Override public TableMetadata getTableMetadata(Session session, TableHandle tableHandle) { @@ -1873,11 +1908,15 @@ public ResolvedFunction resolveOperator(OperatorType operatorType, List argumentTypes; + + private OperatorCacheKey(OperatorType operatorType, List argumentTypes) + { + this.operatorType = requireNonNull(operatorType, "operatorType is null"); + this.argumentTypes = ImmutableList.copyOf(requireNonNull(argumentTypes, "argumentTypes is null")); + } + + public OperatorType getOperatorType() + { + return operatorType; + } + + public List getArgumentTypes() + { + return argumentTypes; + } + + @Override + public int hashCode() + { + return Objects.hash(operatorType, argumentTypes); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (!(obj instanceof OperatorCacheKey)) { + return false; + } + OperatorCacheKey other = (OperatorCacheKey) obj; + return Objects.equals(this.operatorType, other.operatorType) && + Objects.equals(this.argumentTypes, other.argumentTypes); + } + } + + private static class CoercionCacheKey + { + private final OperatorType operatorType; + private final Type fromType; + private final Type toType; + + private CoercionCacheKey(OperatorType operatorType, Type fromType, Type toType) + { + this.operatorType = requireNonNull(operatorType, "operatorType is null"); + this.fromType = requireNonNull(fromType, "fromType is null"); + this.toType = requireNonNull(toType, "toType is null"); + } + + public OperatorType getOperatorType() + { + return operatorType; + } + + public Type getFromType() + { + return fromType; + } + + public Type getToType() + { + return toType; + } + + @Override + public int hashCode() + { + return Objects.hash(operatorType, fromType, toType); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (!(obj instanceof CoercionCacheKey)) { + return false; + } + CoercionCacheKey other = (CoercionCacheKey) obj; + return Objects.equals(this.operatorType, other.operatorType) && + Objects.equals(this.fromType, other.fromType) && + Objects.equals(this.toType, other.toType); + } + } } diff --git a/core/trino-main/src/main/java/io/trino/metadata/TableSchema.java b/core/trino-main/src/main/java/io/trino/metadata/TableSchema.java new file mode 100644 index 000000000000..a29498a1cbd0 --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/metadata/TableSchema.java @@ -0,0 +1,72 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.trino.metadata; + +import io.trino.connector.CatalogName; +import io.trino.spi.connector.ColumnSchema; +import io.trino.spi.connector.ConnectorTableSchema; +import io.trino.spi.connector.SchemaTableName; + +import java.util.List; + +import static com.google.common.collect.MoreCollectors.toOptional; +import static java.util.Objects.requireNonNull; + +public final class TableSchema +{ + private final CatalogName catalogName; + private final ConnectorTableSchema tableSchema; + + public TableSchema(CatalogName catalogName, ConnectorTableSchema tableSchema) + { + requireNonNull(catalogName, "catalog is null"); + requireNonNull(tableSchema, "metadata is null"); + + this.catalogName = catalogName; + this.tableSchema = tableSchema; + } + + public QualifiedObjectName getQualifiedName() + { + return new QualifiedObjectName(catalogName.getCatalogName(), tableSchema.getTable().getSchemaName(), tableSchema.getTable().getTableName()); + } + + public CatalogName getCatalogName() + { + return catalogName; + } + + public ConnectorTableSchema getTableSchema() + { + return tableSchema; + } + + public SchemaTableName getTable() + { + return tableSchema.getTable(); + } + + public List getColumns() + { + return tableSchema.getColumns(); + } + + public ColumnSchema getColumn(String name) + { + return tableSchema.getColumns().stream() + .filter(columnMetadata -> columnMetadata.getName().equals(name)) + .collect(toOptional()) + .orElseThrow(() -> new IllegalArgumentException("Invalid column name: " + name)); + } +} diff --git a/core/trino-main/src/main/java/io/trino/operator/LookupJoinOperator.java b/core/trino-main/src/main/java/io/trino/operator/LookupJoinOperator.java index 4f22d5493845..a6839b0a8881 100644 --- a/core/trino-main/src/main/java/io/trino/operator/LookupJoinOperator.java +++ b/core/trino-main/src/main/java/io/trino/operator/LookupJoinOperator.java @@ -72,6 +72,7 @@ public class LookupJoinOperator implements AdapterWorkProcessorOperator { private final ListenableFuture lookupSourceProviderFuture; + private final boolean waitForBuild; private final PageBuffer pageBuffer; private final WorkProcessor pages; private final SpillingJoinProcessor joinProcessor; @@ -82,6 +83,7 @@ public class LookupJoinOperator List buildOutputTypes, JoinType joinType, boolean outputSingleMatch, + boolean waitForBuild, LookupSourceFactory lookupSourceFactory, JoinProbeFactory joinProbeFactory, Runnable afterClose, @@ -92,8 +94,9 @@ public class LookupJoinOperator Optional> sourcePages) { this.statisticsCounter = new JoinStatisticsCounter(joinType); + this.waitForBuild = waitForBuild; lookupSourceProviderFuture = lookupSourceFactory.createLookupSourceProvider(); - pageBuffer = new PageBuffer(lookupSourceProviderFuture); + pageBuffer = new PageBuffer(); joinProcessor = new SpillingJoinProcessor( processorContext, afterClose, @@ -102,6 +105,7 @@ public class LookupJoinOperator buildOutputTypes, joinType, outputSingleMatch, + waitForBuild, hashGenerator, joinProbeFactory, lookupSourceFactory, @@ -121,7 +125,7 @@ public Optional getOperatorInfo() @Override public boolean needsInput() { - return lookupSourceProviderFuture.isDone() && pageBuffer.isEmpty() && !pageBuffer.isFinished(); + return (!waitForBuild || lookupSourceProviderFuture.isDone()) && pageBuffer.isEmpty() && !pageBuffer.isFinished(); } @Override @@ -518,9 +522,11 @@ private static class SpillingJoinProcessor private final List buildOutputTypes; private final JoinType joinType; private final boolean outputSingleMatch; + private final boolean waitForBuild; private final HashGenerator hashGenerator; private final JoinProbeFactory joinProbeFactory; private final LookupSourceFactory lookupSourceFactory; + private final ListenableFuture lookupSourceProvider; private final JoinStatisticsCounter statisticsCounter; private final PageJoiner sourcePagesJoiner; private final WorkProcessor joinedSourcePages; @@ -544,6 +550,7 @@ private SpillingJoinProcessor( List buildOutputTypes, JoinType joinType, boolean outputSingleMatch, + boolean waitForBuild, HashGenerator hashGenerator, JoinProbeFactory joinProbeFactory, LookupSourceFactory lookupSourceFactory, @@ -559,9 +566,11 @@ private SpillingJoinProcessor( this.buildOutputTypes = requireNonNull(buildOutputTypes, "buildOutputTypes is null"); this.joinType = requireNonNull(joinType, "joinType is null"); this.outputSingleMatch = outputSingleMatch; + this.waitForBuild = waitForBuild; this.hashGenerator = requireNonNull(hashGenerator, "hashGenerator is null"); this.joinProbeFactory = requireNonNull(joinProbeFactory, "joinProbeFactory is null"); this.lookupSourceFactory = requireNonNull(lookupSourceFactory, "lookupSourceFactory is null"); + this.lookupSourceProvider = requireNonNull(lookupSourceProvider, "lookupSourceProvider is null"); this.statisticsCounter = requireNonNull(statisticsCounter, "statisticsCounter is null"); sourcePagesJoiner = new PageJoiner( processorContext, @@ -582,6 +591,12 @@ private SpillingJoinProcessor( @Override public ProcessState> process() { + // wait for build side to be completed before fetching any probe data + // TODO: fix support for probe short-circuit: https://github.com/trinodb/trino/issues/3957 + if (waitForBuild && !lookupSourceProvider.isDone()) { + return ProcessState.blocked(lookupSourceProvider); + } + if (!joinedSourcePages.isFinished()) { return ProcessState.ofResult(joinedSourcePages); } diff --git a/core/trino-main/src/main/java/io/trino/operator/LookupJoinOperatorFactory.java b/core/trino-main/src/main/java/io/trino/operator/LookupJoinOperatorFactory.java index 196a48e50a8d..d8e5d828e4b8 100644 --- a/core/trino-main/src/main/java/io/trino/operator/LookupJoinOperatorFactory.java +++ b/core/trino-main/src/main/java/io/trino/operator/LookupJoinOperatorFactory.java @@ -46,6 +46,7 @@ public class LookupJoinOperatorFactory private final List buildOutputTypes; private final JoinType joinType; private final boolean outputSingleMatch; + private final boolean waitForBuild; private final JoinProbeFactory joinProbeFactory; private final Optional outerOperatorFactoryResult; private final JoinBridgeManager joinBridgeManager; @@ -64,6 +65,7 @@ public LookupJoinOperatorFactory( List buildOutputTypes, JoinType joinType, boolean outputSingleMatch, + boolean waitForBuild, JoinProbeFactory joinProbeFactory, BlockTypeOperators blockTypeOperators, OptionalInt totalOperatorsCount, @@ -77,6 +79,7 @@ public LookupJoinOperatorFactory( this.buildOutputTypes = ImmutableList.copyOf(requireNonNull(buildOutputTypes, "buildOutputTypes is null")); this.joinType = requireNonNull(joinType, "joinType is null"); this.outputSingleMatch = outputSingleMatch; + this.waitForBuild = waitForBuild; this.joinProbeFactory = requireNonNull(joinProbeFactory, "joinProbeFactory is null"); this.joinBridgeManager = lookupSourceFactoryManager; @@ -123,6 +126,7 @@ private LookupJoinOperatorFactory(LookupJoinOperatorFactory other) buildOutputTypes = other.buildOutputTypes; joinType = other.joinType; outputSingleMatch = other.outputSingleMatch; + waitForBuild = other.waitForBuild; joinProbeFactory = other.joinProbeFactory; joinBridgeManager = other.joinBridgeManager; outerOperatorFactoryResult = other.outerOperatorFactoryResult; @@ -193,6 +197,7 @@ public WorkProcessorOperator create(ProcessorContext processorContext, WorkProce buildOutputTypes, joinType, outputSingleMatch, + waitForBuild, lookupSourceFactory, joinProbeFactory, () -> joinBridgeManager.probeOperatorClosed(processorContext.getLifespan()), @@ -215,6 +220,7 @@ public AdapterWorkProcessorOperator createAdapterOperator(ProcessorContext proce buildOutputTypes, joinType, outputSingleMatch, + waitForBuild, lookupSourceFactory, joinProbeFactory, () -> joinBridgeManager.probeOperatorClosed(processorContext.getLifespan()), diff --git a/core/trino-main/src/main/java/io/trino/operator/LookupJoinOperators.java b/core/trino-main/src/main/java/io/trino/operator/LookupJoinOperators.java index 857009866945..ebce8869f22b 100644 --- a/core/trino-main/src/main/java/io/trino/operator/LookupJoinOperators.java +++ b/core/trino-main/src/main/java/io/trino/operator/LookupJoinOperators.java @@ -49,6 +49,7 @@ public OperatorFactory innerJoin( JoinBridgeManager lookupSourceFactory, List probeTypes, boolean outputSingleMatch, + boolean waitForBuild, List probeJoinChannel, OptionalInt probeHashChannel, Optional> probeOutputChannels, @@ -66,6 +67,7 @@ public OperatorFactory innerJoin( probeOutputChannels.orElse(rangeList(probeTypes.size())), JoinType.INNER, outputSingleMatch, + waitForBuild, totalOperatorsCount, partitioningSpillerFactory, blockTypeOperators); @@ -94,6 +96,7 @@ public OperatorFactory probeOuterJoin( probeOutputChannels.orElse(rangeList(probeTypes.size())), JoinType.PROBE_OUTER, outputSingleMatch, + false, totalOperatorsCount, partitioningSpillerFactory, blockTypeOperators); @@ -104,6 +107,7 @@ public OperatorFactory lookupOuterJoin( PlanNodeId planNodeId, JoinBridgeManager lookupSourceFactory, List probeTypes, + boolean waitForBuild, List probeJoinChannel, OptionalInt probeHashChannel, Optional> probeOutputChannels, @@ -121,6 +125,7 @@ public OperatorFactory lookupOuterJoin( probeOutputChannels.orElse(rangeList(probeTypes.size())), JoinType.LOOKUP_OUTER, false, + waitForBuild, totalOperatorsCount, partitioningSpillerFactory, blockTypeOperators); @@ -148,6 +153,7 @@ public OperatorFactory fullOuterJoin( probeOutputChannels.orElse(rangeList(probeTypes.size())), JoinType.FULL_OUTER, false, + false, totalOperatorsCount, partitioningSpillerFactory, blockTypeOperators); @@ -170,6 +176,7 @@ private OperatorFactory createJoinOperatorFactory( List probeOutputChannels, JoinType joinType, boolean outputSingleMatch, + boolean waitForBuild, OptionalInt totalOperatorsCount, PartitioningSpillerFactory partitioningSpillerFactory, BlockTypeOperators blockTypeOperators) @@ -187,6 +194,7 @@ private OperatorFactory createJoinOperatorFactory( lookupSourceFactoryManager.getBuildOutputTypes(), joinType, outputSingleMatch, + waitForBuild, new JoinProbeFactory(probeOutputChannels.stream().mapToInt(i -> i).toArray(), probeJoinChannel, probeHashChannel), blockTypeOperators, totalOperatorsCount, diff --git a/core/trino-main/src/main/java/io/trino/operator/PartitionedLookupSourceFactory.java b/core/trino-main/src/main/java/io/trino/operator/PartitionedLookupSourceFactory.java index 50ca998a707d..19ef6fbe79d8 100644 --- a/core/trino-main/src/main/java/io/trino/operator/PartitionedLookupSourceFactory.java +++ b/core/trino-main/src/main/java/io/trino/operator/PartitionedLookupSourceFactory.java @@ -212,7 +212,7 @@ public void setPartitionSpilledLookupSourceHandle(int partitionIndex, SpilledLoo lock.writeLock().lock(); try { - if (destroyed.isDone()) { + if (partitionsNoLongerNeeded.isDone()) { spilledLookupSourceHandle.dispose(); return; } @@ -302,6 +302,14 @@ public ListenableFuture>> finishPr try { if (!spillingInfo.hasSpilled()) { finishedProbeOperators++; + if (lookupJoinsCount.isPresent()) { + checkState(finishedProbeOperators <= lookupJoinsCount.getAsInt(), "%s probe operators finished out of %s declared", finishedProbeOperators, lookupJoinsCount.getAsInt()); + if (finishedProbeOperators == lookupJoinsCount.getAsInt()) { + // We can dispose partitions now since right outer is not supported with spill and lookupJoinsCount should be absent + freePartitions(); + } + } + return immediateFuture(new PartitionedConsumption<>( 1, emptyList(), @@ -322,7 +330,7 @@ public ListenableFuture>> finishPr finishedProbeOperators++; if (finishedProbeOperators == operatorsCount) { - // We can dispose partitions now since as right outer is not supported with spill + // We can dispose partitions now since right outer is not supported with spill freePartitions(); verify(!partitionedConsumption.isDone()); partitionedConsumption.set(new PartitionedConsumption<>( diff --git a/core/trino-main/src/main/java/io/trino/operator/scalar/DateTimeFunctions.java b/core/trino-main/src/main/java/io/trino/operator/scalar/DateTimeFunctions.java index 188e68462eeb..b6a96ad32876 100644 --- a/core/trino-main/src/main/java/io/trino/operator/scalar/DateTimeFunctions.java +++ b/core/trino-main/src/main/java/io/trino/operator/scalar/DateTimeFunctions.java @@ -24,7 +24,6 @@ import io.trino.spi.function.LiteralParameters; import io.trino.spi.function.ScalarFunction; import io.trino.spi.function.SqlType; -import io.trino.spi.type.LongTimestamp; import io.trino.spi.type.LongTimestampWithTimeZone; import io.trino.spi.type.StandardTypes; import io.trino.spi.type.TimeZoneKey; @@ -55,7 +54,6 @@ import static io.trino.spi.type.Timestamps.NANOSECONDS_PER_SECOND; import static io.trino.spi.type.UnscaledDecimal128Arithmetic.rescale; import static io.trino.spi.type.UnscaledDecimal128Arithmetic.unscaledDecimalToBigInteger; -import static io.trino.type.DateTimes.MICROSECONDS_PER_SECOND; import static io.trino.type.DateTimes.PICOSECONDS_PER_NANOSECOND; import static io.trino.type.DateTimes.PICOSECONDS_PER_SECOND; import static io.trino.type.DateTimes.scaleEpochMillisToMicros; @@ -121,10 +119,11 @@ public static Slice currentTimeZone(ConnectorSession session) } @ScalarFunction("from_unixtime") - @SqlType("timestamp(3)") - public static long fromUnixTime(@SqlType(StandardTypes.DOUBLE) double unixTime) + @SqlType("timestamp(3) with time zone") + public static long fromUnixTime(ConnectorSession session, @SqlType(StandardTypes.DOUBLE) double unixTime) { - return Math.round(unixTime * MICROSECONDS_PER_SECOND); + // TODO (https://github.com/trinodb/trino/issues/5781) + return packDateTimeWithZone(Math.round(unixTime * 1000), session.getTimeZoneKey()); } @ScalarFunction("from_unixtime") @@ -155,9 +154,10 @@ public static final class FromUnixtimeNanosDecimal private FromUnixtimeNanosDecimal() {} @LiteralParameters({"p", "s"}) - @SqlType("timestamp(9)") - public static LongTimestamp fromLong(@LiteralParameter("s") long scale, @SqlType("decimal(p, s)") Slice unixTimeNanos) + @SqlType("timestamp(9) with time zone") + public static LongTimestampWithTimeZone fromLong(@LiteralParameter("s") long scale, ConnectorSession session, @SqlType("decimal(p, s)") Slice unixTimeNanos) { + // TODO (https://github.com/trinodb/trino/issues/5781) BigInteger unixTimeNanosInt = unscaledDecimalToBigInteger(rescale(unixTimeNanos, -(int) scale)); long epochSeconds = unixTimeNanosInt.divide(BigInteger.valueOf(NANOSECONDS_PER_SECOND)).longValue(); long nanosOfSecond = unixTimeNanosInt.remainder(BigInteger.valueOf(NANOSECONDS_PER_SECOND)).longValue(); @@ -167,27 +167,28 @@ public static LongTimestamp fromLong(@LiteralParameter("s") long scale, @SqlType epochSeconds -= 1; picosOfSecond += PICOSECONDS_PER_SECOND; } - return DateTimes.longTimestamp(epochSeconds, picosOfSecond); + return DateTimes.longTimestampWithTimeZone(epochSeconds, picosOfSecond, session.getTimeZoneKey().getZoneId()); } @LiteralParameters({"p", "s"}) - @SqlType("timestamp(9)") - public static LongTimestamp fromShort(@LiteralParameter("s") long scale, @SqlType("decimal(p, s)") long unixTimeNanos) + @SqlType("timestamp(9) with time zone") + public static LongTimestampWithTimeZone fromShort(@LiteralParameter("s") long scale, ConnectorSession session, @SqlType("decimal(p, s)") long unixTimeNanos) { + // TODO (https://github.com/trinodb/trino/issues/5781) long roundedUnixTimeNanos = MathFunctions.Round.roundShort(scale, unixTimeNanos); - return fromUnixtimeNanosLong(roundedUnixTimeNanos); + return fromUnixtimeNanosLong(session, roundedUnixTimeNanos); } } @ScalarFunction("from_unixtime_nanos") - @SqlType("timestamp(9)") - public static LongTimestamp fromUnixtimeNanosLong(@SqlType(StandardTypes.BIGINT) long unixTimeNanos) + @SqlType("timestamp(9) with time zone") + public static LongTimestampWithTimeZone fromUnixtimeNanosLong(ConnectorSession session, @SqlType(StandardTypes.BIGINT) long unixTimeNanos) { long epochSeconds = floorDiv(unixTimeNanos, NANOSECONDS_PER_SECOND); long nanosOfSecond = floorMod(unixTimeNanos, NANOSECONDS_PER_SECOND); long picosOfSecond = nanosOfSecond * PICOSECONDS_PER_NANOSECOND; - return DateTimes.longTimestamp(epochSeconds, picosOfSecond); + return DateTimes.longTimestampWithTimeZone(epochSeconds, picosOfSecond, session.getTimeZoneKey().getZoneId()); } @ScalarFunction("to_iso8601") diff --git a/core/trino-main/src/main/java/io/trino/operator/scalar/timestamp/ToUnixTime.java b/core/trino-main/src/main/java/io/trino/operator/scalar/timestamp/ToUnixTime.java deleted file mode 100644 index 22db5bd4b81e..000000000000 --- a/core/trino-main/src/main/java/io/trino/operator/scalar/timestamp/ToUnixTime.java +++ /dev/null @@ -1,43 +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 io.trino.operator.scalar.timestamp; - -import io.trino.spi.function.LiteralParameters; -import io.trino.spi.function.ScalarFunction; -import io.trino.spi.function.SqlType; -import io.trino.spi.type.LongTimestamp; -import io.trino.spi.type.StandardTypes; - -import static io.trino.type.DateTimes.MICROSECONDS_PER_SECOND; -import static io.trino.type.DateTimes.PICOSECONDS_PER_SECOND; - -@ScalarFunction("to_unixtime") -public final class ToUnixTime -{ - private ToUnixTime() {} - - @LiteralParameters("p") - @SqlType(StandardTypes.DOUBLE) - public static double toUnixTime(@SqlType("timestamp(p)") long timestamp) - { - return timestamp * 1.0 / MICROSECONDS_PER_SECOND; - } - - @LiteralParameters("p") - @SqlType(StandardTypes.DOUBLE) - public static double toUnixTime(@SqlType("timestamp(p)") LongTimestamp timestamp) - { - return timestamp.getEpochMicros() * 1.0 / MICROSECONDS_PER_SECOND + timestamp.getPicosOfMicro() * 1.0 / PICOSECONDS_PER_SECOND; - } -} diff --git a/core/trino-main/src/main/java/io/trino/server/PluginManager.java b/core/trino-main/src/main/java/io/trino/server/PluginManager.java index 8fc144b40976..c04d601f84e1 100644 --- a/core/trino-main/src/main/java/io/trino/server/PluginManager.java +++ b/core/trino-main/src/main/java/io/trino/server/PluginManager.java @@ -42,6 +42,7 @@ import java.net.URL; import java.util.List; +import java.util.Optional; import java.util.ServiceLoader; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Supplier; @@ -67,7 +68,7 @@ public class PluginManager private final MetadataManager metadataManager; private final ResourceGroupManager resourceGroupManager; private final AccessControlManager accessControlManager; - private final PasswordAuthenticatorManager passwordAuthenticatorManager; + private final Optional passwordAuthenticatorManager; private final CertificateAuthenticatorManager certificateAuthenticatorManager; private final EventListenerManager eventListenerManager; private final GroupProviderManager groupProviderManager; @@ -82,7 +83,7 @@ public PluginManager( MetadataManager metadataManager, ResourceGroupManager resourceGroupManager, AccessControlManager accessControlManager, - PasswordAuthenticatorManager passwordAuthenticatorManager, + Optional passwordAuthenticatorManager, CertificateAuthenticatorManager certificateAuthenticatorManager, EventListenerManager eventListenerManager, GroupProviderManager groupProviderManager, @@ -191,10 +192,12 @@ private void installPluginInternal(Plugin plugin, Supplier duplicat accessControlManager.addSystemAccessControlFactory(accessControlFactory); } - for (PasswordAuthenticatorFactory authenticatorFactory : plugin.getPasswordAuthenticatorFactories()) { - log.info("Registering password authenticator %s", authenticatorFactory.getName()); - passwordAuthenticatorManager.addPasswordAuthenticatorFactory(authenticatorFactory); - } + passwordAuthenticatorManager.ifPresent(authenticationManager -> { + for (PasswordAuthenticatorFactory authenticatorFactory : plugin.getPasswordAuthenticatorFactories()) { + log.info("Registering password authenticator %s", authenticatorFactory.getName()); + authenticationManager.addPasswordAuthenticatorFactory(authenticatorFactory); + } + }); for (CertificateAuthenticatorFactory authenticatorFactory : plugin.getCertificateAuthenticatorFactories()) { log.info("Registering certificate authenticator %s", authenticatorFactory.getName()); diff --git a/core/trino-main/src/main/java/io/trino/server/Server.java b/core/trino-main/src/main/java/io/trino/server/Server.java index 559430890508..6bed0e26e676 100644 --- a/core/trino-main/src/main/java/io/trino/server/Server.java +++ b/core/trino-main/src/main/java/io/trino/server/Server.java @@ -17,7 +17,10 @@ import com.google.common.base.StandardSystemProperty; import com.google.common.collect.ImmutableList; import com.google.inject.Injector; +import com.google.inject.Key; import com.google.inject.Module; +import com.google.inject.TypeLiteral; +import com.google.inject.util.Types; import io.airlift.bootstrap.ApplicationConfigurationException; import io.airlift.bootstrap.Bootstrap; import io.airlift.discovery.client.Announcer; @@ -56,6 +59,7 @@ import java.nio.file.Path; import java.nio.file.Paths; import java.util.List; +import java.util.Optional; import java.util.Set; import static com.google.common.collect.ImmutableSet.toImmutableSet; @@ -124,7 +128,8 @@ private void doStart(String trinoVersion) injector.getInstance(SessionPropertyDefaults.class).loadConfigurationManager(); injector.getInstance(ResourceGroupManager.class).loadConfigurationManager(); injector.getInstance(AccessControlManager.class).loadSystemAccessControl(); - injector.getInstance(PasswordAuthenticatorManager.class).loadPasswordAuthenticator(); + injector.getInstance(optionalKey(PasswordAuthenticatorManager.class)) + .ifPresent(PasswordAuthenticatorManager::loadPasswordAuthenticator); injector.getInstance(EventListenerManager.class).loadEventListeners(); injector.getInstance(GroupProviderManager.class).loadConfiguredGroupProvider(); injector.getInstance(CertificateAuthenticatorManager.class).loadCertificateAuthenticator(); @@ -152,6 +157,12 @@ private void doStart(String trinoVersion) } } + @SuppressWarnings("unchecked") + private static Key> optionalKey(Class type) + { + return Key.get((TypeLiteral>) TypeLiteral.get(Types.newParameterizedType(Optional.class, type))); + } + private static void addMessages(StringBuilder output, String type, List messages) { if (messages.isEmpty()) { diff --git a/core/trino-main/src/main/java/io/trino/server/remotetask/HttpRemoteTask.java b/core/trino-main/src/main/java/io/trino/server/remotetask/HttpRemoteTask.java index 5f6dc43155e1..6318e5c67ba7 100644 --- a/core/trino-main/src/main/java/io/trino/server/remotetask/HttpRemoteTask.java +++ b/core/trino-main/src/main/java/io/trino/server/remotetask/HttpRemoteTask.java @@ -159,6 +159,7 @@ public final class HttpRemoteTask private final PartitionedSplitCountTracker partitionedSplitCountTracker; + private final AtomicBoolean started = new AtomicBoolean(false); private final AtomicBoolean aborting = new AtomicBoolean(false); public HttpRemoteTask( @@ -318,6 +319,7 @@ public void start() { try (SetThreadName ignored = new SetThreadName("HttpRemoteTask-%s", taskId)) { // to start we just need to trigger an update + started.set(true); scheduleUpdate(); dynamicFiltersFetcher.start(); @@ -493,9 +495,9 @@ private synchronized void processTaskUpdate(TaskInfo newValue, List pendingSourceSplitCount -= removed; } } - updateSplitQueueSpace(); - + // Update node level split tracker before split queue space to ensure it's up to date before waking up the scheduler partitionedSplitCountTracker.setPartitionedSplitCount(getPartitionedSplitCount()); + updateSplitQueueSpace(); } private void updateTaskInfo(TaskInfo taskInfo) @@ -513,7 +515,7 @@ private synchronized void sendUpdate() { TaskStatus taskStatus = getTaskStatus(); // don't update if the task hasn't been started yet or if it is already finished - if (!needsUpdate.get() || taskStatus.getState().isDone()) { + if (!started.get() || !needsUpdate.get() || taskStatus.getState().isDone()) { return; } diff --git a/core/trino-main/src/main/java/io/trino/server/security/AbstractBearerAuthenticator.java b/core/trino-main/src/main/java/io/trino/server/security/AbstractBearerAuthenticator.java index 6eef6513945d..99cbdd4aebf0 100644 --- a/core/trino-main/src/main/java/io/trino/server/security/AbstractBearerAuthenticator.java +++ b/core/trino-main/src/main/java/io/trino/server/security/AbstractBearerAuthenticator.java @@ -43,24 +43,12 @@ protected AbstractBearerAuthenticator(String principalField, UserMapping userMap public Identity authenticate(ContainerRequestContext request) throws AuthenticationException { - List headers = request.getHeaders().get(AUTHORIZATION); - if (headers == null || headers.size() == 0) { - throw needAuthentication(request, null); - } - if (headers.size() > 1) { - throw new IllegalArgumentException(format("Multiple %s headers detected: %s, where only single %s header is supported", AUTHORIZATION, headers, AUTHORIZATION)); - } - - String header = headers.get(0); - int space = header.indexOf(' '); - if ((space < 0) || !header.substring(0, space).equalsIgnoreCase("bearer")) { - throw needAuthentication(request, null); - } - String token = header.substring(space + 1).trim(); - if (token.isEmpty()) { - throw needAuthentication(request, null); - } + return authenticate(request, extractToken(request)); + } + public Identity authenticate(ContainerRequestContext request, String token) + throws AuthenticationException + { try { Jws claimsJws = parseClaimsJws(token); String principal = claimsJws.getBody().get(principalField, String.class); @@ -80,6 +68,29 @@ public Identity authenticate(ContainerRequestContext request) } } + public String extractToken(ContainerRequestContext request) + throws AuthenticationException + { + List headers = request.getHeaders().get(AUTHORIZATION); + if (headers == null || headers.size() == 0) { + throw needAuthentication(request, null); + } + if (headers.size() > 1) { + throw new IllegalArgumentException(format("Multiple %s headers detected: %s, where only single %s header is supported", AUTHORIZATION, headers, AUTHORIZATION)); + } + + String header = headers.get(0); + int space = header.indexOf(' '); + if ((space < 0) || !header.substring(0, space).equalsIgnoreCase("bearer")) { + throw needAuthentication(request, null); + } + String token = header.substring(space + 1).trim(); + if (token.isEmpty()) { + throw needAuthentication(request, null); + } + return token; + } + protected abstract Jws parseClaimsJws(String jws); protected abstract AuthenticationException needAuthentication(ContainerRequestContext request, String message); diff --git a/core/trino-main/src/main/java/io/trino/server/security/AuthenticationFilter.java b/core/trino-main/src/main/java/io/trino/server/security/AuthenticationFilter.java index 65d4eae0f16d..35f84ce16a9d 100644 --- a/core/trino-main/src/main/java/io/trino/server/security/AuthenticationFilter.java +++ b/core/trino-main/src/main/java/io/trino/server/security/AuthenticationFilter.java @@ -24,9 +24,11 @@ import javax.ws.rs.container.ContainerRequestContext; import javax.ws.rs.container.ContainerRequestFilter; +import java.util.Arrays; import java.util.LinkedHashSet; import java.util.List; import java.util.Set; +import java.util.stream.Stream; import static com.google.common.base.Preconditions.checkArgument; import static io.trino.server.ServletSecurityUtils.sendWwwAuthenticate; @@ -86,10 +88,17 @@ else if (insecureAuthenticationOverHttpAllowed) { authenticatedIdentity = authenticator.authenticate(request); } catch (AuthenticationException e) { - if (e.getMessage() != null) { - messages.add(e.getMessage()); - } - e.getAuthenticateHeader().ifPresent(authenticateHeaders::add); + // Some authenticators (e.g. password) nest multiple internal authenticators. + // Exceptions from additional failed login attempts are suppressed in the first exception + Stream.concat(Stream.of(e), Arrays.stream(e.getSuppressed())) + .filter(ex -> ex instanceof AuthenticationException) + .map(AuthenticationException.class::cast) + .forEach(ex -> { + if (ex.getMessage() != null) { + messages.add(ex.getMessage()); + } + ex.getAuthenticateHeader().ifPresent(authenticateHeaders::add); + }); continue; } diff --git a/core/trino-main/src/main/java/io/trino/server/security/PasswordAuthenticator.java b/core/trino-main/src/main/java/io/trino/server/security/PasswordAuthenticator.java index 95f780e7b9e7..bf4dd094b657 100644 --- a/core/trino-main/src/main/java/io/trino/server/security/PasswordAuthenticator.java +++ b/core/trino-main/src/main/java/io/trino/server/security/PasswordAuthenticator.java @@ -21,6 +21,7 @@ import java.security.Principal; +import static com.google.common.base.Verify.verify; import static io.trino.server.security.BasicAuthCredentials.extractBasicAuthCredentials; import static io.trino.server.security.UserMapping.createUserMapping; import static java.util.Objects.requireNonNull; @@ -47,22 +48,34 @@ public Identity authenticate(ContainerRequestContext request) { BasicAuthCredentials basicAuthCredentials = extractBasicAuthCredentials(request) .orElseThrow(() -> needAuthentication(null)); - try { - Principal principal = authenticatorManager.getAuthenticator().createAuthenticatedPrincipal( - basicAuthCredentials.getUser(), - basicAuthCredentials.getPassword() - .orElseThrow(() -> new AuthenticationException("Malformed credentials: password is empty"))); - String authenticatedUser = userMapping.mapUser(principal.toString()); - return Identity.forUser(authenticatedUser) - .withPrincipal(principal) - .build(); - } - catch (UserMappingException | AccessDeniedException e) { - throw needAuthentication(e.getMessage()); - } - catch (RuntimeException e) { - throw new RuntimeException("Authentication error", e); + String user = basicAuthCredentials.getUser(); + String password = basicAuthCredentials.getPassword() + .orElseThrow(() -> new AuthenticationException("Malformed credentials: password is empty")); + + AuthenticationException exception = null; + for (io.trino.spi.security.PasswordAuthenticator authenticator : authenticatorManager.getAuthenticators()) { + try { + Principal principal = authenticator.createAuthenticatedPrincipal(user, password); + String authenticatedUser = userMapping.mapUser(principal.toString()); + return Identity.forUser(authenticatedUser) + .withPrincipal(principal) + .build(); + } + catch (UserMappingException | AccessDeniedException e) { + if (exception == null) { + exception = needAuthentication(e.getMessage()); + } + else { + exception.addSuppressed(needAuthentication(e.getMessage())); + } + } + catch (RuntimeException e) { + throw new RuntimeException("Authentication error", e); + } } + + verify(exception != null, "exception not set"); + throw exception; } private static AuthenticationException needAuthentication(String message) diff --git a/core/trino-main/src/main/java/io/trino/server/security/PasswordAuthenticatorConfig.java b/core/trino-main/src/main/java/io/trino/server/security/PasswordAuthenticatorConfig.java index 1ed99ecc2601..6fb163eb1575 100644 --- a/core/trino-main/src/main/java/io/trino/server/security/PasswordAuthenticatorConfig.java +++ b/core/trino-main/src/main/java/io/trino/server/security/PasswordAuthenticatorConfig.java @@ -13,16 +13,27 @@ */ package io.trino.server.security; +import com.google.common.base.Splitter; +import com.google.common.collect.ImmutableList; import io.airlift.configuration.Config; +import io.airlift.configuration.ConfigDescription; import io.airlift.configuration.validation.FileExists; +import javax.validation.constraints.NotEmpty; +import javax.validation.constraints.NotNull; + import java.io.File; +import java.util.List; import java.util.Optional; +import static com.google.common.collect.ImmutableList.toImmutableList; + public class PasswordAuthenticatorConfig { + private static final Splitter SPLITTER = Splitter.on(',').trimResults().omitEmptyStrings(); private Optional userMappingPattern = Optional.empty(); private Optional userMappingFile = Optional.empty(); + private List passwordAuthenticatorFiles = ImmutableList.of(new File("etc/password-authenticator.properties")); public Optional getUserMappingPattern() { @@ -47,4 +58,21 @@ public PasswordAuthenticatorConfig setUserMappingFile(File userMappingFile) this.userMappingFile = Optional.ofNullable(userMappingFile); return this; } + + @NotNull + @NotEmpty(message = "At least one password authenticator config file is required") + public List<@FileExists File> getPasswordAuthenticatorFiles() + { + return passwordAuthenticatorFiles; + } + + @Config("password-authenticator.config-files") + @ConfigDescription("Ordered list of password authenticator config files") + public PasswordAuthenticatorConfig setPasswordAuthenticatorFiles(String passwordAuthenticatorFiles) + { + this.passwordAuthenticatorFiles = SPLITTER.splitToList(passwordAuthenticatorFiles).stream() + .map(File::new) + .collect(toImmutableList()); + return this; + } } diff --git a/core/trino-main/src/main/java/io/trino/server/security/PasswordAuthenticatorManager.java b/core/trino-main/src/main/java/io/trino/server/security/PasswordAuthenticatorManager.java index b11c39b94b51..20575ebfb14c 100644 --- a/core/trino-main/src/main/java/io/trino/server/security/PasswordAuthenticatorManager.java +++ b/core/trino-main/src/main/java/io/trino/server/security/PasswordAuthenticatorManager.java @@ -14,13 +14,18 @@ package io.trino.server.security; import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.inject.Inject; import io.airlift.log.Logger; import io.trino.spi.security.PasswordAuthenticator; import io.trino.spi.security.PasswordAuthenticatorFactory; import java.io.File; +import java.io.IOException; +import java.io.UncheckedIOException; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicBoolean; @@ -36,12 +41,20 @@ public class PasswordAuthenticatorManager { private static final Logger log = Logger.get(PasswordAuthenticatorManager.class); - private static final File CONFIG_FILE = new File("etc/password-authenticator.properties"); private static final String NAME_PROPERTY = "password-authenticator.name"; + private final List configFiles; private final AtomicBoolean required = new AtomicBoolean(); private final Map factories = new ConcurrentHashMap<>(); - private final AtomicReference authenticator = new AtomicReference<>(); + private final AtomicReference> authenticators = new AtomicReference<>(); + + @Inject + public PasswordAuthenticatorManager(PasswordAuthenticatorConfig config) + { + requireNonNull(config, "config is null"); + this.configFiles = ImmutableList.copyOf(config.getPasswordAuthenticatorFiles()); + checkArgument(!configFiles.isEmpty(), "password authenticator files list is empty"); + } public void setRequired() { @@ -56,18 +69,31 @@ public void addPasswordAuthenticatorFactory(PasswordAuthenticatorFactory factory public boolean isLoaded() { - return authenticator.get() != null; + return authenticators.get() != null; } public void loadPasswordAuthenticator() - throws Exception { if (!required.get()) { return; } - File configFile = CONFIG_FILE.getAbsoluteFile(); - Map properties = new HashMap<>(loadPropertiesFrom(configFile.getPath())); + ImmutableList.Builder authenticators = ImmutableList.builder(); + for (File configFile : configFiles) { + authenticators.add(loadAuthenticator(configFile.getAbsoluteFile())); + } + this.authenticators.set(authenticators.build()); + } + + private PasswordAuthenticator loadAuthenticator(File configFile) + { + Map properties; + try { + properties = new HashMap<>(loadPropertiesFrom(configFile.getPath())); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } String name = properties.remove(NAME_PROPERTY); checkState(!isNullOrEmpty(name), "Password authenticator configuration %s does not contain '%s'", configFile, NAME_PROPERTY); @@ -77,23 +103,21 @@ public void loadPasswordAuthenticator() PasswordAuthenticatorFactory factory = factories.get(name); checkState(factory != null, "Password authenticator '%s' is not registered", name); - PasswordAuthenticator authenticator = factory.create(ImmutableMap.copyOf(properties)); - this.authenticator.set(requireNonNull(authenticator, "authenticator is null")); - log.info("-- Loaded password authenticator %s --", name); + return factory.create(ImmutableMap.copyOf(properties)); } - public PasswordAuthenticator getAuthenticator() + public List getAuthenticators() { - checkState(isLoaded(), "authenticator was not loaded"); - return authenticator.get(); + checkState(isLoaded(), "authenticators were not loaded"); + return authenticators.get(); } @VisibleForTesting - public void setAuthenticator(PasswordAuthenticator authenticator) + public void setAuthenticators(PasswordAuthenticator... authenticators) { - if (!this.authenticator.compareAndSet(null, authenticator)) { - throw new IllegalStateException("authenticator already loaded"); + if (!this.authenticators.compareAndSet(null, ImmutableList.copyOf(authenticators))) { + throw new IllegalStateException("authenticators already loaded"); } } } diff --git a/core/trino-main/src/main/java/io/trino/server/security/ServerSecurityModule.java b/core/trino-main/src/main/java/io/trino/server/security/ServerSecurityModule.java index ba729ef62eea..b2539d3247ae 100644 --- a/core/trino-main/src/main/java/io/trino/server/security/ServerSecurityModule.java +++ b/core/trino-main/src/main/java/io/trino/server/security/ServerSecurityModule.java @@ -61,7 +61,7 @@ protected void setup(Binder binder) .internalOnlyResource(DynamicAnnouncementResource.class) .internalOnlyResource(StoreResource.class); - binder.bind(PasswordAuthenticatorManager.class).in(Scopes.SINGLETON); + newOptionalBinder(binder, PasswordAuthenticatorManager.class); binder.bind(CertificateAuthenticatorManager.class).in(Scopes.SINGLETON); insecureHttpAuthenticationDefaults(); @@ -73,7 +73,10 @@ protected void setup(Binder binder) configBinder(certificateBinder).bindConfig(CertificateConfig.class); })); installAuthenticator("kerberos", KerberosAuthenticator.class, KerberosConfig.class); - installAuthenticator("password", PasswordAuthenticator.class, PasswordAuthenticatorConfig.class); + install(authenticatorModule("password", PasswordAuthenticator.class, used -> { + configBinder(binder).bindConfig(PasswordAuthenticatorConfig.class); + binder.bind(PasswordAuthenticatorManager.class).in(Scopes.SINGLETON); + })); install(authenticatorModule("jwt", JwtAuthenticator.class, new JwtAuthenticatorSupportModule())); install(authenticatorModule("oauth2", OAuth2Authenticator.class, new OAuth2AuthenticationSupportModule())); diff --git a/core/trino-main/src/main/java/io/trino/server/security/oauth2/OAuth2CallbackResource.java b/core/trino-main/src/main/java/io/trino/server/security/oauth2/OAuth2CallbackResource.java index 6ceb412b91aa..be65f8e6e3c9 100644 --- a/core/trino-main/src/main/java/io/trino/server/security/oauth2/OAuth2CallbackResource.java +++ b/core/trino-main/src/main/java/io/trino/server/security/oauth2/OAuth2CallbackResource.java @@ -88,20 +88,13 @@ public Response callback( // Note: the Web UI may be disabled, so REST requests can not redirect to a success or error page inside of the Web UI if (error != null) { - LOG.debug( - "OAuth server returned an error: error=%s, error_description=%s, error_uri=%s, state=%s", - error, - errorDescription, - errorUri, - state); - - passErrorToTokenExchange( - authId, - "OAuth server returned an error: error=%s, error_description=%s, error_uri=%s, state=%s", - error, - errorDescription, - errorUri, - state); + LOG.debug("OAuth server returned an error: error=%s, error_description=%s, error_uri=%s, state=%s", error, errorDescription, errorUri, state); + + if (tokenExchange.isPresent() && authId.isPresent()) { + tokenExchange.get().setTokenExchangeError( + authId.get(), + format("OAuth server returned an error: error=%s, error_description=%s, error_uri=%s, state=%s", error, errorDescription, errorUri, state)); + } return Response.ok() .entity(service.getCallbackErrorHtml(error)) .build(); @@ -117,8 +110,9 @@ public Response callback( } catch (ChallengeFailedException | RuntimeException e) { LOG.debug(e, "Authentication response could not be verified: state=%s", state); - - passErrorToTokenExchange(authId, "Authentication response could not be verified: state=%s", state); + if (tokenExchange.isPresent() && authId.isPresent()) { + tokenExchange.get().setTokenExchangeError(authId.get(), format("Authentication response could not be verified: state=%s", state)); + } return Response.ok() .entity(service.getInternalFailureHtml("Authentication response could not be verified")) .build(); @@ -146,12 +140,4 @@ public Response callback( } return builder.build(); } - - private void passErrorToTokenExchange(Optional authId, String format, String... args) - { - if (tokenExchange.isEmpty() || authId.isEmpty()) { - return; - } - tokenExchange.orElseThrow().setTokenExchangeError(authId.orElseThrow(), format(format, args)); - } } diff --git a/core/trino-main/src/main/java/io/trino/server/testing/TestingTrinoServer.java b/core/trino-main/src/main/java/io/trino/server/testing/TestingTrinoServer.java index 41bf994aea11..42bb7fb6cd6b 100644 --- a/core/trino-main/src/main/java/io/trino/server/testing/TestingTrinoServer.java +++ b/core/trino-main/src/main/java/io/trino/server/testing/TestingTrinoServer.java @@ -69,6 +69,7 @@ import io.trino.server.security.ServerSecurityModule; import io.trino.spi.Plugin; import io.trino.spi.QueryId; +import io.trino.spi.eventlistener.EventListener; import io.trino.spi.security.GroupProvider; import io.trino.spi.security.SystemAccessControl; import io.trino.split.PageSourceManager; @@ -194,7 +195,8 @@ private TestingTrinoServer( Optional discoveryUri, Module additionalModule, Optional baseDataDir, - List systemAccessControls) + List systemAccessControls, + List eventListeners) { this.coordinator = coordinator; @@ -317,6 +319,9 @@ private TestingTrinoServer( accessControl.setSystemAccessControls(systemAccessControls); + EventListenerManager eventListenerManager = injector.getInstance(EventListenerManager.class); + eventListeners.forEach(eventListenerManager::addEventListener); + announcer.forceAnnounce(); refreshNodes(); @@ -593,6 +598,7 @@ public static class Builder private Module additionalModule = EMPTY_MODULE; private Optional baseDataDir = Optional.empty(); private List systemAccessControls = ImmutableList.of(); + private List eventListeners = ImmutableList.of(); public Builder setCoordinator(boolean coordinator) { @@ -636,6 +642,12 @@ public Builder setSystemAccessControls(List systemAccessCon return this; } + public Builder setEventListeners(List eventListeners) + { + this.eventListeners = ImmutableList.copyOf(requireNonNull(eventListeners, "eventListeners is null")); + return this; + } + public TestingTrinoServer build() { return new TestingTrinoServer( @@ -645,7 +657,8 @@ public TestingTrinoServer build() discoveryUri, additionalModule, baseDataDir, - systemAccessControls); + systemAccessControls, + eventListeners); } } } diff --git a/core/trino-main/src/main/java/io/trino/server/ui/FormUiAuthenticatorModule.java b/core/trino-main/src/main/java/io/trino/server/ui/FormUiAuthenticatorModule.java index 7b0b208a2fec..cf7f6cff65bb 100644 --- a/core/trino-main/src/main/java/io/trino/server/ui/FormUiAuthenticatorModule.java +++ b/core/trino-main/src/main/java/io/trino/server/ui/FormUiAuthenticatorModule.java @@ -17,6 +17,7 @@ import com.google.inject.Key; import com.google.inject.Module; import io.trino.server.security.Authenticator; +import io.trino.server.security.PasswordAuthenticatorConfig; import io.trino.server.security.PasswordAuthenticatorManager; import static com.google.inject.Scopes.SINGLETON; @@ -37,10 +38,11 @@ public FormUiAuthenticatorModule(boolean usePasswordManager) @Override public void configure(Binder binder) { - binder.bind(PasswordAuthenticatorManager.class).in(SINGLETON); binder.bind(FormWebUiAuthenticationFilter.class).in(SINGLETON); binder.bind(WebUiAuthenticationFilter.class).to(FormWebUiAuthenticationFilter.class).in(SINGLETON); if (usePasswordManager) { + binder.bind(PasswordAuthenticatorManager.class).in(SINGLETON); + configBinder(binder).bindConfig(PasswordAuthenticatorConfig.class); binder.bind(FormAuthenticator.class).to(PasswordManagerFormAuthenticator.class).in(SINGLETON); } else { diff --git a/core/trino-main/src/main/java/io/trino/server/ui/PasswordManagerFormAuthenticator.java b/core/trino-main/src/main/java/io/trino/server/ui/PasswordManagerFormAuthenticator.java index be068ea58e70..6bea02bb9831 100644 --- a/core/trino-main/src/main/java/io/trino/server/ui/PasswordManagerFormAuthenticator.java +++ b/core/trino-main/src/main/java/io/trino/server/ui/PasswordManagerFormAuthenticator.java @@ -21,6 +21,8 @@ import javax.inject.Inject; +import java.util.List; + import static java.util.Objects.requireNonNull; public class PasswordManagerFormAuthenticator @@ -66,17 +68,20 @@ public boolean isValidCredential(String username, String password, boolean secur return insecureAuthenticationOverHttpAllowed && password == null; } - PasswordAuthenticator authenticator = passwordAuthenticatorManager.getAuthenticator(); - try { - authenticator.createAuthenticatedPrincipal(username, password); - return true; - } - catch (AccessDeniedException e) { - return false; - } - catch (RuntimeException e) { - log.debug(e, "Error authenticating user for Web UI"); - return false; + List authenticators = passwordAuthenticatorManager.getAuthenticators(); + for (PasswordAuthenticator authenticator : authenticators) { + try { + authenticator.createAuthenticatedPrincipal(username, password); + return true; + } + catch (AccessDeniedException e) { + // Try another one + } + catch (RuntimeException e) { + log.debug(e, "Error authenticating user for Web UI"); + } } + + return false; } } diff --git a/core/trino-main/src/main/java/io/trino/sql/analyzer/Analysis.java b/core/trino-main/src/main/java/io/trino/sql/analyzer/Analysis.java index 46e28434568c..94f96381c0dd 100644 --- a/core/trino-main/src/main/java/io/trino/sql/analyzer/Analysis.java +++ b/core/trino-main/src/main/java/io/trino/sql/analyzer/Analysis.java @@ -13,11 +13,13 @@ */ package io.trino.sql.analyzer; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ArrayListMultimap; -import com.google.common.collect.HashMultimap; import com.google.common.collect.HashMultiset; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; import com.google.common.collect.ListMultimap; import com.google.common.collect.Multimap; import com.google.common.collect.Multiset; @@ -32,6 +34,7 @@ import io.trino.spi.connector.ColumnHandle; import io.trino.spi.connector.ColumnMetadata; import io.trino.spi.connector.ConnectorTableMetadata; +import io.trino.spi.eventlistener.ColumnDetail; import io.trino.spi.eventlistener.ColumnInfo; import io.trino.spi.eventlistener.RoutineInfo; import io.trino.spi.eventlistener.TableInfo; @@ -121,9 +124,6 @@ public class Analysis // a map of users to the columns per table that they access private final Map>> tableColumnReferences = new LinkedHashMap<>(); - // Track referenced fields from source relation node - private final Multimap, Field> referencedFields = HashMultimap.create(); - private final Map, List> aggregates = new LinkedHashMap<>(); private final Map, List> orderByAggregates = new LinkedHashMap<>(); private final Map, GroupingSetAnalysis> groupingSets = new LinkedHashMap<>(); @@ -194,6 +194,8 @@ public class Analysis // row id field for update/delete queries private final Map, FieldReference> rowIdField = new LinkedHashMap<>(); + private final Multimap originColumnDetails = ArrayListMultimap.create(); + private final Multimap, Field> fieldLineage = ArrayListMultimap.create(); public Analysis(@Nullable Statement root, Map, Expression> parameters, boolean isDescribe) { @@ -216,14 +218,14 @@ public Optional getTarget() { return target.map(target -> { QualifiedObjectName name = target.getName(); - return new Output(name.getCatalogName(), name.getSchemaName(), name.getObjectName()); + return new Output(name.getCatalogName(), name.getSchemaName(), name.getObjectName(), target.getColumns()); }); } - public void setUpdateType(String updateType, QualifiedObjectName targetName, Optional targetTable) + public void setUpdateType(String updateType, QualifiedObjectName targetName, Optional
targetTable, Optional> targetColumns) { this.updateType = updateType; - this.target = Optional.of(new UpdateTarget(targetName, targetTable)); + this.target = Optional.of(new UpdateTarget(targetName, targetTable, targetColumns)); } public void resetUpdateType() @@ -850,11 +852,6 @@ public void addEmptyColumnReferencesForTable(AccessControl accessControl, Identi tableColumnReferences.computeIfAbsent(accessControlInfo, k -> new LinkedHashMap<>()).computeIfAbsent(table, k -> new HashSet<>()); } - public void addReferencedFields(Multimap, Field> references) - { - referencedFields.putAll(references); - } - public Map>> getTableColumnReferences() { return tableColumnReferences; @@ -965,6 +962,28 @@ public List getRoutines() .collect(toImmutableList()); } + public void addSourceColumns(Field field, Set sourceColumn) + { + originColumnDetails.putAll(field, sourceColumn); + } + + public Set getSourceColumns(Field field) + { + return ImmutableSet.copyOf(originColumnDetails.get(field)); + } + + public void addExpressionFields(Expression expression, Collection fields) + { + fieldLineage.putAll(NodeRef.of(expression), fields); + } + + public Set getExpressionSourceColumns(Expression expression) + { + return fieldLineage.get(NodeRef.of(expression)).stream() + .flatMap(field -> getSourceColumns(field).stream()) + .collect(toImmutableSet()); + } + public void setRowIdField(Table table, FieldReference field) { rowIdField.put(NodeRef.of(table), field); @@ -1487,6 +1506,56 @@ public Scope getAccessControlScope() } } + public static class SourceColumn + { + private final QualifiedObjectName tableName; + private final String columnName; + + @JsonCreator + public SourceColumn(@JsonProperty("tableName") QualifiedObjectName tableName, @JsonProperty("columnName") String columnName) + { + this.tableName = requireNonNull(tableName, "tableName is null"); + this.columnName = requireNonNull(columnName, "columnName is null"); + } + + @JsonProperty + public QualifiedObjectName getTableName() + { + return tableName; + } + + @JsonProperty + public String getColumnName() + { + return columnName; + } + + public ColumnDetail getColumnDetail() + { + return new ColumnDetail(tableName.getCatalogName(), tableName.getSchemaName(), tableName.getObjectName(), columnName); + } + + @Override + public int hashCode() + { + return Objects.hash(tableName, columnName); + } + + @Override + public boolean equals(Object obj) + { + if (obj == this) { + return true; + } + if ((obj == null) || (getClass() != obj.getClass())) { + return false; + } + SourceColumn entry = (SourceColumn) obj; + return Objects.equals(tableName, entry.tableName) && + Objects.equals(columnName, entry.columnName); + } + } + private static class RoutineEntry { private final ResolvedFunction function; @@ -1513,11 +1582,13 @@ private static class UpdateTarget { private final QualifiedObjectName name; private final Optional
table; + private final Optional> columns; - public UpdateTarget(QualifiedObjectName name, Optional
table) + public UpdateTarget(QualifiedObjectName name, Optional
table, Optional> columns) { this.name = requireNonNull(name, "name is null"); this.table = requireNonNull(table, "table is null"); + this.columns = requireNonNull(columns, "columns is null").map(ImmutableList::copyOf); } public QualifiedObjectName getName() @@ -1529,5 +1600,10 @@ public Optional
getTable() { return table; } + + public Optional> getColumns() + { + return columns; + } } } diff --git a/core/trino-main/src/main/java/io/trino/sql/analyzer/ExpressionAnalyzer.java b/core/trino-main/src/main/java/io/trino/sql/analyzer/ExpressionAnalyzer.java index a73d722d24d5..2eba8f272590 100644 --- a/core/trino-main/src/main/java/io/trino/sql/analyzer/ExpressionAnalyzer.java +++ b/core/trino-main/src/main/java/io/trino/sql/analyzer/ExpressionAnalyzer.java @@ -256,6 +256,7 @@ public class ExpressionAnalyzer private final CorrelationSupport correlationSupport; private final Function getPreanalyzedType; private final Function getResolvedWindow; + private final List sourceFields = new ArrayList<>(); public ExpressionAnalyzer( Metadata metadata, @@ -401,6 +402,11 @@ public Multimap, Field> getReferencedFields() return referencedFields; } + public List getSourceFields() + { + return sourceFields; + } + private class Visitor extends StackableAstVisitor { @@ -507,6 +513,8 @@ private Type handleResolvedField(Expression node, ResolvedField resolvedField, S tableColumnReferences.put(field.getOriginTable().get(), field.getOriginColumnName().get()); } + sourceFields.add(field); + fieldId.getRelationId() .getSourceNode() .ifPresent(source -> referencedFields.put(NodeRef.of(source), field)); @@ -1571,6 +1579,8 @@ else if (previousNode instanceof QuantifiedComparisonExpression) { scalarSubqueries.add(NodeRef.of(node)); } + sourceFields.add(queryScope.getRelationType().getFieldByIndex(0)); + Type type = getOnlyElement(queryScope.getRelationType().getVisibleFields()).getType(); return setExpressionType(node, type); } @@ -1973,6 +1983,7 @@ public static ExpressionAnalysis analyzeExpression( analyzer.analyze(expression, scope); updateAnalysis(analysis, analyzer, session, accessControl); + analysis.addExpressionFields(expression, analyzer.getSourceFields()); return new ExpressionAnalysis( analyzer.getExpressionTypes(), @@ -2030,7 +2041,6 @@ private static void updateAnalysis(Analysis analysis, ExpressionAnalyzer analyze analysis.addColumnReferences(analyzer.getColumnReferences()); analysis.addLambdaArgumentReferences(analyzer.getLambdaArgumentReferences()); analysis.addTableColumnReferences(accessControl, session.getIdentity(), analyzer.getTableColumnReferences()); - analysis.addReferencedFields(analyzer.getReferencedFields()); } public static ExpressionAnalyzer create( diff --git a/core/trino-main/src/main/java/io/trino/sql/analyzer/FeaturesConfig.java b/core/trino-main/src/main/java/io/trino/sql/analyzer/FeaturesConfig.java index 8f23911c8169..b61d9a716e0a 100644 --- a/core/trino-main/src/main/java/io/trino/sql/analyzer/FeaturesConfig.java +++ b/core/trino-main/src/main/java/io/trino/sql/analyzer/FeaturesConfig.java @@ -906,6 +906,7 @@ public int getMaxRecursionDepth() } @Config("max-recursion-depth") + @ConfigDescription("Maximum recursion depth for recursive common table expression") public FeaturesConfig setMaxRecursionDepth(int maxRecursionDepth) { this.maxRecursionDepth = maxRecursionDepth; diff --git a/core/trino-main/src/main/java/io/trino/sql/analyzer/Output.java b/core/trino-main/src/main/java/io/trino/sql/analyzer/Output.java index 91def9382de8..abb4adcea85b 100644 --- a/core/trino-main/src/main/java/io/trino/sql/analyzer/Output.java +++ b/core/trino-main/src/main/java/io/trino/sql/analyzer/Output.java @@ -15,10 +15,13 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; import javax.annotation.concurrent.Immutable; +import java.util.List; import java.util.Objects; +import java.util.Optional; import static java.util.Objects.requireNonNull; @@ -28,16 +31,19 @@ public final class Output private final String catalogName; private final String schema; private final String table; + private final Optional> columns; @JsonCreator public Output( @JsonProperty("catalogName") String catalogName, @JsonProperty("schema") String schema, - @JsonProperty("table") String table) + @JsonProperty("table") String table, + @JsonProperty("columns") Optional> columns) { this.catalogName = requireNonNull(catalogName, "catalogName is null"); this.schema = requireNonNull(schema, "schema is null"); this.table = requireNonNull(table, "table is null"); + this.columns = requireNonNull(columns, "columns is null").map(ImmutableList::copyOf); } @JsonProperty @@ -58,6 +64,12 @@ public String getTable() return table; } + @JsonProperty + public Optional> getColumns() + { + return columns; + } + @Override public boolean equals(Object o) { @@ -70,12 +82,13 @@ public boolean equals(Object o) Output output = (Output) o; return Objects.equals(catalogName, output.catalogName) && Objects.equals(schema, output.schema) && - Objects.equals(table, output.table); + Objects.equals(table, output.table) && + Objects.equals(columns, output.columns); } @Override public int hashCode() { - return Objects.hash(catalogName, schema, table); + return Objects.hash(catalogName, schema, table, columns); } } diff --git a/core/trino-main/src/main/java/io/trino/sql/analyzer/OutputColumn.java b/core/trino-main/src/main/java/io/trino/sql/analyzer/OutputColumn.java new file mode 100644 index 000000000000..7682e79be713 --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/sql/analyzer/OutputColumn.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 io.trino.sql.analyzer; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableSet; +import io.trino.execution.Column; +import io.trino.sql.analyzer.Analysis.SourceColumn; + +import javax.annotation.concurrent.Immutable; + +import java.util.Objects; +import java.util.Set; + +import static java.util.Objects.requireNonNull; + +@Immutable +public final class OutputColumn +{ + private final Column column; + private final Set sourceColumns; + + @JsonCreator + public OutputColumn(@JsonProperty("column") Column column, @JsonProperty("sourceColumns") Set sourceColumns) + { + this.column = requireNonNull(column, "column is null"); + this.sourceColumns = ImmutableSet.copyOf(requireNonNull(sourceColumns, "sourceColumns is null")); + } + + @JsonProperty + public Column getColumn() + { + return column; + } + + @JsonProperty + public Set getSourceColumns() + { + return sourceColumns; + } + + @Override + public int hashCode() + { + return Objects.hash(column, sourceColumns); + } + + @Override + public boolean equals(Object obj) + { + if (obj == this) { + return true; + } + if ((obj == null) || (getClass() != obj.getClass())) { + return false; + } + OutputColumn entry = (OutputColumn) obj; + return Objects.equals(column, entry.column) && + Objects.equals(sourceColumns, entry.sourceColumns); + } +} diff --git a/core/trino-main/src/main/java/io/trino/sql/analyzer/StatementAnalyzer.java b/core/trino-main/src/main/java/io/trino/sql/analyzer/StatementAnalyzer.java index 6f7945018479..a5251e702a74 100644 --- a/core/trino-main/src/main/java/io/trino/sql/analyzer/StatementAnalyzer.java +++ b/core/trino-main/src/main/java/io/trino/sql/analyzer/StatementAnalyzer.java @@ -20,8 +20,10 @@ import com.google.common.collect.ImmutableMultimap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; +import com.google.common.collect.Streams; import io.trino.Session; import io.trino.connector.CatalogName; +import io.trino.execution.Column; import io.trino.execution.warnings.WarningCollector; import io.trino.metadata.FunctionKind; import io.trino.metadata.FunctionMetadata; @@ -32,6 +34,7 @@ import io.trino.metadata.ResolvedFunction; import io.trino.metadata.TableHandle; import io.trino.metadata.TableMetadata; +import io.trino.metadata.TableSchema; import io.trino.security.AccessControl; import io.trino.security.AllowAllAccessControl; import io.trino.security.ViewAccessControl; @@ -40,6 +43,7 @@ import io.trino.spi.connector.CatalogSchemaName; import io.trino.spi.connector.ColumnHandle; import io.trino.spi.connector.ColumnMetadata; +import io.trino.spi.connector.ColumnSchema; import io.trino.spi.connector.ConnectorMaterializedViewDefinition; import io.trino.spi.connector.ConnectorTableMetadata; import io.trino.spi.connector.ConnectorViewDefinition; @@ -61,6 +65,7 @@ import io.trino.sql.analyzer.Analysis.GroupingSetAnalysis; import io.trino.sql.analyzer.Analysis.ResolvedWindow; import io.trino.sql.analyzer.Analysis.SelectExpression; +import io.trino.sql.analyzer.Analysis.SourceColumn; import io.trino.sql.analyzer.Analysis.UnnestAnalysis; import io.trino.sql.analyzer.Scope.AsteriskedIdentifierChainBasis; import io.trino.sql.parser.ParsingException; @@ -397,8 +402,6 @@ protected Scope visitInsert(Insert insert, Optional scope) // analyze the query that creates the data Scope queryScope = analyze(insert.getQuery(), createScope(scope)); - analysis.setUpdateType("INSERT", targetTable, Optional.empty()); - // verify the insert destination columns match the query Optional targetTableHandle = metadata.getTableHandle(session, targetTable); if (targetTableHandle.isEmpty()) { @@ -410,20 +413,20 @@ protected Scope visitInsert(Insert insert, Optional scope) throw semanticException(NOT_SUPPORTED, insert, "Insert into table with a row filter is not supported"); } - TableMetadata tableMetadata = metadata.getTableMetadata(session, targetTableHandle.get()); + TableSchema tableSchema = metadata.getTableSchema(session, targetTableHandle.get()); - List columns = tableMetadata.getColumns().stream() + List columns = tableSchema.getColumns().stream() .filter(column -> !column.isHidden()) .collect(toImmutableList()); - for (ColumnMetadata column : columns) { + for (ColumnSchema column : columns) { if (!accessControl.getColumnMasks(session.toSecurityContext(), targetTable, column.getName(), column.getType()).isEmpty()) { throw semanticException(NOT_SUPPORTED, insert, "Insert into table with column masks is not supported"); } } List tableColumns = columns.stream() - .map(ColumnMetadata::getName) + .map(ColumnSchema::getName) .collect(toImmutableList()); // analyze target table layout, table columns should contain all partition columns @@ -462,7 +465,7 @@ protected Scope visitInsert(Insert insert, Optional scope) newTableLayout)); List tableTypes = insertColumns.stream() - .map(insertColumn -> tableMetadata.getColumn(insertColumn).getType()) + .map(insertColumn -> tableSchema.getColumn(insertColumn).getType()) .collect(toImmutableList()); List queryTypes = queryScope.getRelationType().getVisibleFields().stream() @@ -477,6 +480,22 @@ protected Scope visitInsert(Insert insert, Optional scope) Joiner.on(", ").join(queryTypes)); } + Stream columnStream = Streams.zip( + insertColumns.stream(), + tableTypes.stream() + .map(Type::toString), + Column::new); + + analysis.setUpdateType( + "INSERT", + targetTable, + Optional.empty(), + Optional.of(Streams.zip( + columnStream, + queryScope.getRelationType().getVisibleFields().stream(), + (column, field) -> new OutputColumn(column, analysis.getSourceColumns(field))) + .collect(toImmutableList()))); + return createAndAssignScope(insert, scope, Field.newUnqualified("rows", BIGINT)); } @@ -490,7 +509,7 @@ protected Scope visitRefreshMaterializedView(RefreshMaterializedView refreshMate throw semanticException(TABLE_NOT_FOUND, refreshMaterializedView, "Materialized view '%s' does not exist", name); } - Optional storageName = getMaterializedViewStorageTableName(name); + Optional storageName = getMaterializedViewStorageTableName(optionalView.get(), name); if (storageName.isEmpty()) { throw semanticException(TABLE_NOT_FOUND, refreshMaterializedView, "Storage Table '%s' for materialized view '%s' does not exist", storageName, name); @@ -502,20 +521,13 @@ protected Scope visitRefreshMaterializedView(RefreshMaterializedView refreshMate Query query = parseView(optionalView.get().getOriginalSql(), name, refreshMaterializedView); Scope queryScope = process(query, scope); - analysis.setUpdateType("REFRESH MATERIALIZED VIEW", targetTable, Optional.empty()); - // verify the insert destination columns match the query Optional targetTableHandle = metadata.getTableHandle(session, targetTable); if (targetTableHandle.isEmpty()) { throw semanticException(TABLE_NOT_FOUND, refreshMaterializedView, "Table '%s' does not exist", targetTable); } - if (targetTableHandle.isPresent() && metadata.getMaterializedViewFreshness(session, name).isMaterializedViewFresh()) { - analysis.setSkipMaterializedViewRefresh(true); - } - else { - analysis.setSkipMaterializedViewRefresh(false); - } + analysis.setSkipMaterializedViewRefresh(metadata.getMaterializedViewFreshness(session, name).isMaterializedViewFresh()); TableMetadata tableMetadata = metadata.getTableMetadata(session, targetTableHandle.get()); List insertColumns = tableMetadata.getColumns().stream() @@ -545,6 +557,22 @@ protected Scope visitRefreshMaterializedView(RefreshMaterializedView refreshMate "Query: [" + Joiner.on(", ").join(queryTypes) + "]"); } + Stream columns = Streams.zip( + insertColumns.stream(), + tableTypes.stream() + .map(Type::toString), + Column::new); + + analysis.setUpdateType( + "REFRESH MATERIALIZED VIEW", + targetTable, + Optional.empty(), + Optional.of(Streams.zip( + columns, + queryScope.getRelationType().getVisibleFields().stream(), + (column, field) -> new OutputColumn(column, analysis.getSourceColumns(field))) + .collect(toImmutableList()))); + return createAndAssignScope(refreshMaterializedView, scope, Field.newUnqualified("rows", BIGINT)); } @@ -642,7 +670,7 @@ protected Scope visitDelete(Delete node, Optional scope) Scope tableScope = analyzer.analyzeForUpdate(table, scope, UpdateKind.DELETE); node.getWhere().ifPresent(where -> analyzeWhere(node, tableScope, where)); - analysis.setUpdateType("DELETE", tableName, Optional.of(table)); + analysis.setUpdateType("DELETE", tableName, Optional.of(table), Optional.empty()); return createAndAssignScope(node, scope, Field.newUnqualified("rows", BIGINT)); } @@ -651,7 +679,7 @@ protected Scope visitDelete(Delete node, Optional scope) protected Scope visitAnalyze(Analyze node, Optional scope) { QualifiedObjectName tableName = createQualifiedObjectName(session, node, node.getTableName()); - analysis.setUpdateType("ANALYZE", tableName, Optional.empty()); + analysis.setUpdateType("ANALYZE", tableName, Optional.empty(), Optional.empty()); // verify the target table exists and it's not a view if (metadata.getView(session, tableName).isPresent()) { @@ -696,7 +724,6 @@ protected Scope visitCreateTableAsSelect(CreateTableAsSelect node, Optional targetTableHandle = metadata.getTableHandle(session, targetTable); if (targetTableHandle.isPresent()) { @@ -707,6 +734,7 @@ protected Scope visitCreateTableAsSelect(CreateTableAsSelect node, Optional columns = ImmutableList.builder(); // analyze target table columns and column aliases + ImmutableList.Builder outputColumns = ImmutableList.builder(); if (node.getColumnAliases().isPresent()) { validateColumnAliases(node.getColumnAliases().get(), queryScope.getRelationType().getVisibleFieldCount()); @@ -730,7 +759,9 @@ protected Scope visitCreateTableAsSelect(CreateTableAsSelect node, Optional new ColumnMetadata(field.getName().get(), field.getType())) .collect(toImmutableList())); + queryScope.getRelationType().getVisibleFields().stream() + .map(this::createOutputColumn) + .forEach(outputColumns::add); } // create target table metadata @@ -783,6 +817,12 @@ protected Scope visitCreateTableAsSelect(CreateTableAsSelect node, Optional scope) { QualifiedObjectName viewName = createQualifiedObjectName(session, node, node.getName()); - analysis.setUpdateType("CREATE VIEW", viewName, Optional.empty()); // analyze the query that creates the view StatementAnalyzer analyzer = new StatementAnalyzer(analysis, metadata, sqlParser, groupProvider, accessControl, session, warningCollector, CorrelationSupport.ALLOWED); @@ -801,6 +840,14 @@ protected Scope visitCreateView(CreateView node, Optional scope) validateColumns(node, queryScope.getRelationType()); + analysis.setUpdateType( + "CREATE VIEW", + viewName, + Optional.empty(), + Optional.of(queryScope.getRelationType().getVisibleFields().stream() + .map(this::createOutputColumn) + .collect(toImmutableList()))); + return createAndAssignScope(node, scope); } @@ -975,7 +1022,6 @@ protected Scope visitCall(Call node, Optional scope) protected Scope visitCreateMaterializedView(CreateMaterializedView node, Optional scope) { QualifiedObjectName viewName = createQualifiedObjectName(session, node, node.getName()); - analysis.setUpdateType("CREATE MATERIALIZED VIEW", viewName, Optional.empty()); if (node.isReplace() && node.isNotExists()) { throw semanticException(NOT_SUPPORTED, node, "'CREATE OR REPLACE' and 'IF NOT EXISTS' clauses can not be used together"); @@ -994,6 +1040,15 @@ protected Scope visitCreateMaterializedView(CreateMaterializedView node, Optiona validateColumns(node, queryScope.getRelationType()); + analysis.setUpdateType( + "CREATE MATERIALIZED VIEW", + viewName, + Optional.empty(), + Optional.of( + queryScope.getRelationType().getVisibleFields().stream() + .map(this::createOutputColumn) + .collect(toImmutableList()))); + return createAndAssignScope(node, scope); } @@ -1119,7 +1174,7 @@ protected Scope visitQuery(Query node, Optional scope) @Override protected Scope visitUnnest(Unnest node, Optional scope) { - ImmutableMap.Builder, List> mappings = ImmutableMap., List>builder(); + ImmutableMap.Builder, List> mappings = ImmutableMap.builder(); ImmutableList.Builder outputFields = ImmutableList.builder(); for (Expression expression : node.getExpressions()) { @@ -1170,14 +1225,9 @@ protected Scope visitLateral(Lateral node, Optional scope) return createAndAssignScope(node, scope, queryScope.getRelationType()); } - private Optional getMaterializedViewStorageTableName(QualifiedObjectName name) + private Optional getMaterializedViewStorageTableName(ConnectorMaterializedViewDefinition viewDefinition, QualifiedObjectName name) { - Optional optionalView = metadata.getMaterializedView(session, name); - if (optionalView.isEmpty()) { - return Optional.empty(); - } - - String storageTable = optionalView.get().getStorageTable(); + String storageTable = viewDefinition.getStorageTable(); if (storageTable == null || storageTable.isEmpty()) { return Optional.empty(); } @@ -1217,7 +1267,7 @@ protected Scope visitTable(Table table, Optional scope) if (optionalMaterializedView.isPresent()) { if (metadata.getMaterializedViewFreshness(session, name).isMaterializedViewFresh()) { // If materialized view is current, answer the query using the storage table - Optional storageName = getMaterializedViewStorageTableName(name); + Optional storageName = getMaterializedViewStorageTableName(optionalMaterializedView.get(), name); if (storageName.isPresent()) { tableHandle = metadata.getTableHandle(session, createQualifiedObjectName(session, table, storageName.get())); } @@ -1245,12 +1295,12 @@ protected Scope visitTable(Table table, Optional scope) } throw semanticException(TABLE_NOT_FOUND, table, "Table '%s' does not exist", name); } - TableMetadata tableMetadata = metadata.getTableMetadata(session, tableHandle.get()); + TableSchema tableSchema = metadata.getTableSchema(session, tableHandle.get()); Map columnHandles = metadata.getColumnHandles(session, tableHandle.get()); // TODO: discover columns lazily based on where they are needed (to support connectors that can't enumerate all tables) ImmutableList.Builder fields = ImmutableList.builder(); - for (ColumnMetadata column : tableMetadata.getColumns()) { + for (ColumnSchema column : tableSchema.getColumns()) { Field field = Field.newQualified( table.getName(), Optional.of(column.getName()), @@ -1263,6 +1313,7 @@ protected Scope visitTable(Table table, Optional scope) ColumnHandle columnHandle = columnHandles.get(column.getName()); checkArgument(columnHandle != null, "Unknown field %s", field); analysis.setColumn(field, columnHandle); + analysis.addSourceColumns(field, ImmutableSet.of(new SourceColumn(name, column.getName()))); } if (updateKind.isPresent()) { @@ -1394,48 +1445,18 @@ private Scope createScopeForCommonTableExpression(Table table, Optional s return createAndAssignScope(table, scope, fields); } - private Scope createScopeForView(Table table, QualifiedObjectName name, Optional scope, ConnectorViewDefinition view) + private Scope createScopeForMaterializedView(Table table, QualifiedObjectName name, Optional scope, ConnectorMaterializedViewDefinition view) { - Statement statement = analysis.getStatement(); - if (statement instanceof CreateView) { - CreateView viewStatement = (CreateView) statement; - QualifiedObjectName viewNameFromStatement = createQualifiedObjectName(session, viewStatement, viewStatement.getName()); - if (viewStatement.isReplace() && viewNameFromStatement.equals(name)) { - throw semanticException(VIEW_IS_RECURSIVE, table, "Statement would create a recursive view"); - } - } - if (analysis.hasTableInView(table)) { - throw semanticException(VIEW_IS_RECURSIVE, table, "View is recursive"); - } - - Query query = parseView(view.getOriginalSql(), name, table); - analysis.registerNamedQuery(table, query); - analysis.registerTableForView(table); - RelationType descriptor = analyzeView(query, name, view.getCatalog(), view.getSchema(), view.getOwner(), table); - analysis.unregisterTableForView(); - - checkViewStaleness(view.getColumns(), descriptor.getVisibleFields(), name, table) - .ifPresent(explanation -> { throw semanticException(VIEW_IS_STALE, table, "View '%s' is stale or in invalid state: %s", name, explanation); }); - - // Derive the type of the view from the stored definition, not from the analysis of the underlying query. - // This is needed in case the underlying table(s) changed and the query in the view now produces types that - // are implicitly coercible to the declared view types. - List outputFields = view.getColumns().stream() - .map(column -> Field.newQualified( - table.getName(), - Optional.of(column.getName()), - getViewColumnType(column, name, table), - false, - Optional.of(name), - Optional.of(column.getName()), - false)) - .collect(toImmutableList()); - - analysis.addRelationCoercion(table, outputFields.stream().map(Field::getType).toArray(Type[]::new)); - - analyzeFiltersAndMasks(table, name, Optional.empty(), outputFields, session.getIdentity().getUser()); - - return createAndAssignScope(table, scope, outputFields); + checkArgument(view.getOwner().isPresent(), "owner must be present"); + return createScopeForView( + table, + name, + scope, + view.getOriginalSql(), + view.getCatalog(), + view.getSchema(), + view.getOwner(), + translateMaterializedViewColumns(view.getColumns())); } private List translateMaterializedViewColumns(List materializedViewColumns) @@ -1447,9 +1468,29 @@ private List translateMaterializedViewColumn return viewColumns; } - private Scope createScopeForMaterializedView(Table table, QualifiedObjectName name, Optional scope, ConnectorMaterializedViewDefinition view) + private Scope createScopeForView(Table table, QualifiedObjectName name, Optional scope, ConnectorViewDefinition view) + { + return createScopeForView(table, name, scope, view.getOriginalSql(), view.getCatalog(), view.getSchema(), view.getOwner(), view.getColumns()); + } + + private Scope createScopeForView( + Table table, + QualifiedObjectName name, + Optional scope, + String originalSql, + Optional catalog, + Optional schema, + Optional owner, + List columns) { Statement statement = analysis.getStatement(); + if (statement instanceof CreateView) { + CreateView viewStatement = (CreateView) statement; + QualifiedObjectName viewNameFromStatement = createQualifiedObjectName(session, viewStatement, viewStatement.getName()); + if (viewStatement.isReplace() && viewNameFromStatement.equals(name)) { + throw semanticException(VIEW_IS_RECURSIVE, table, "Statement would create a recursive view"); + } + } if (statement instanceof CreateMaterializedView) { CreateMaterializedView viewStatement = (CreateMaterializedView) statement; QualifiedObjectName viewNameFromStatement = createQualifiedObjectName(session, viewStatement, viewStatement.getName()); @@ -1458,23 +1499,22 @@ private Scope createScopeForMaterializedView(Table table, QualifiedObjectName na } } if (analysis.hasTableInView(table)) { - throw semanticException(VIEW_IS_RECURSIVE, table, "Materialized View is recursive"); + throw semanticException(VIEW_IS_RECURSIVE, table, "View is recursive"); } - Query query = parseView(view.getOriginalSql(), name, table); + Query query = parseView(originalSql, name, table); analysis.registerNamedQuery(table, query); analysis.registerTableForView(table); - RelationType descriptor = analyzeView(query, name, view.getCatalog(), view.getSchema(), view.getOwner(), table); + RelationType descriptor = analyzeView(query, name, catalog, schema, owner, table); analysis.unregisterTableForView(); - List viewColumns = translateMaterializedViewColumns(view.getColumns()); - checkViewStaleness(viewColumns, descriptor.getVisibleFields(), name, table) - .ifPresent(explanation -> { throw semanticException(VIEW_IS_STALE, table, "Materialized View '%s' is stale or in invalid state: %s", name, explanation); }); + checkViewStaleness(columns, descriptor.getVisibleFields(), name, table) + .ifPresent(explanation -> { throw semanticException(VIEW_IS_STALE, table, "View '%s' is stale or in invalid state: %s", name, explanation); }); - // Derive the type of the materialized view from the stored definition, not from the analysis of the underlying query. - // This is needed in case the underlying table(s) changed and the query in the materialized view now produces types that - // are implicitly coercible to the declared materialized view types. - List outputFields = viewColumns.stream() + // Derive the type of the view from the stored definition, not from the analysis of the underlying query. + // This is needed in case the underlying table(s) changed and the query in the view now produces types that + // are implicitly coercible to the declared view types. + List outputFields = columns.stream() .map(column -> Field.newQualified( table.getName(), Optional.of(column.getName()), @@ -1489,6 +1529,7 @@ private Scope createScopeForMaterializedView(Table table, QualifiedObjectName na analyzeFiltersAndMasks(table, name, Optional.empty(), outputFields, session.getIdentity().getUser()); + outputFields.forEach(field -> analysis.addSourceColumns(field, ImmutableSet.of(new SourceColumn(name, field.getName().orElseThrow())))); return createAndAssignScope(table, scope, outputFields); } @@ -1879,7 +1920,6 @@ protected Scope visitUpdate(Update update, Optional scope) List updatedColumns = allColumns.stream() .filter(column -> assignmentTargets.contains(column.getName())) .collect(toImmutableList()); - analysis.setUpdateType("UPDATE", tableName, Optional.of(table)); analysis.setUpdatedColumns(updatedColumns); // Analyzer checks for select permissions but UPDATE has a separate permission, so disable access checks @@ -1930,6 +1970,14 @@ protected Scope visitUpdate(Update update, Optional scope) analysis.recordSubqueries(update, analyses.get(index)); } + analysis.setUpdateType( + "UPDATE", + tableName, + Optional.of(table), + Optional.of(updatedColumns.stream() + .map(column -> new OutputColumn(new Column(column.getName(), column.getType().toString()), ImmutableSet.of())) + .collect(toImmutableList()))); + return createAndAssignScope(update, scope, Field.newUnqualified("rows", BIGINT)); } @@ -2498,7 +2546,9 @@ private Scope computeAndAssignOutputScope(QuerySpecification node, Optional fromReferences = findReferences(from, withQuery.getName()); - if (fromReferences.size() == 0) { + if (fromReferences.isEmpty()) { throw semanticException(INVALID_RECURSIVE_REFERENCE, stepReferences.get(0), "recursive reference outside of FROM clause of the step relation of recursion"); } @@ -3629,6 +3688,11 @@ private Scope.Builder scopeBuilder(Optional parentScope) return scopeBuilder; } + + private OutputColumn createOutputColumn(Field field) + { + return new OutputColumn(new Column(field.getName().orElseThrow(), field.getType().toString()), analysis.getSourceColumns(field)); + } } private Session createViewSession(Optional catalog, Optional schema, Identity identity, SqlPath path) diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/DistributedExecutionPlanner.java b/core/trino-main/src/main/java/io/trino/sql/planner/DistributedExecutionPlanner.java index 5f130f4ed359..0ca6bc3844d0 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/DistributedExecutionPlanner.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/DistributedExecutionPlanner.java @@ -19,8 +19,8 @@ import io.trino.Session; import io.trino.execution.TableInfo; import io.trino.metadata.Metadata; -import io.trino.metadata.TableMetadata; import io.trino.metadata.TableProperties; +import io.trino.metadata.TableSchema; import io.trino.operator.StageExecutionDescriptor; import io.trino.server.DynamicFilterService; import io.trino.spi.connector.DynamicFilter; @@ -149,9 +149,9 @@ private StageExecutionPlan doPlan(SubPlan root, Session session, ImmutableList.B private TableInfo getTableInfo(TableScanNode node, Session session) { - TableMetadata tableMetadata = metadata.getTableMetadata(session, node.getTable()); + TableSchema tableSchema = metadata.getTableSchema(session, node.getTable()); TableProperties tableProperties = metadata.getTableProperties(session, node.getTable()); - return new TableInfo(tableMetadata.getQualifiedName(), tableProperties.getPredicate()); + return new TableInfo(tableSchema.getQualifiedName(), tableProperties.getPredicate()); } private final class Visitor diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/InputExtractor.java b/core/trino-main/src/main/java/io/trino/sql/planner/InputExtractor.java index 39ea0222ad4a..8db19fcf1447 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/InputExtractor.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/InputExtractor.java @@ -63,7 +63,7 @@ private static Column createColumn(ColumnMetadata columnMetadata) private Input createInput(Session session, TableHandle table, Set columns, PlanFragmentId fragmentId, PlanNodeId planNodeId) { - SchemaTableName schemaTable = metadata.getTableMetadata(session, table).getTable(); + SchemaTableName schemaTable = metadata.getTableSchema(session, table).getTable(); Optional inputMetadata = metadata.getInfo(session, table); return new Input(table.getCatalogName().getCatalogName(), schemaTable.getSchemaName(), schemaTable.getTableName(), inputMetadata, ImmutableList.copyOf(columns), fragmentId, planNodeId); } diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/LocalExecutionPlanner.java b/core/trino-main/src/main/java/io/trino/sql/planner/LocalExecutionPlanner.java index 767b7407c971..49f700050df1 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/LocalExecutionPlanner.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/LocalExecutionPlanner.java @@ -1743,6 +1743,7 @@ public PhysicalOperation visitIndexJoin(IndexJoinNode node, LocalExecutionPlanCo lookupSourceFactoryManager, probeSource.getTypes(), false, + false, probeChannels, probeHashChannel, Optional.empty(), @@ -2110,11 +2111,23 @@ private PhysicalOperation createLookupJoin( PhysicalOperation probeSource = probeNode.accept(this, context); // Plan build - boolean spillEnabled = isSpillEnabled(session) && node.isSpillable().orElseThrow(() -> new IllegalArgumentException("spillable not yet set")); + boolean buildOuter = node.getType() == RIGHT || node.getType() == FULL; + boolean spillEnabled = isSpillEnabled(session) + && node.isSpillable().orElseThrow(() -> new IllegalArgumentException("spillable not yet set")) + && probeSource.getPipelineExecutionStrategy() == UNGROUPED_EXECUTION + && !buildOuter; JoinBridgeManager lookupSourceFactory = createLookupSourceFactory(node, buildNode, buildSymbols, buildHashSymbol, probeSource, context, spillEnabled, localDynamicFilters); - OperatorFactory operator = createLookupJoin(node, probeSource, probeSymbols, probeHashSymbol, lookupSourceFactory, context, spillEnabled); + OperatorFactory operator = createLookupJoin( + node, + probeSource, + probeSymbols, + probeHashSymbol, + lookupSourceFactory, + context, + spillEnabled, + !localDynamicFilters.isEmpty()); ImmutableMap.Builder outputMappings = ImmutableMap.builder(); List outputSymbols = node.getOutputSymbols(); @@ -2219,7 +2232,7 @@ private JoinBridgeManager createLookupSourceFact searchFunctionFactories, 10_000, pagesIndexFactory, - spillEnabled && !buildOuter && partitionCount > 1, + spillEnabled && partitionCount > 1, singleStreamSpillerFactory); context.addDriverFactory( @@ -2316,15 +2329,19 @@ private OperatorFactory createLookupJoin( Optional probeHashSymbol, JoinBridgeManager lookupSourceFactoryManager, LocalExecutionPlanContext context, - boolean spillEnabled) + boolean spillEnabled, + boolean consumedLocalDynamicFilters) { List probeTypes = probeSource.getTypes(); List probeOutputChannels = ImmutableList.copyOf(getChannelsForSymbols(node.getLeftOutputSymbols(), probeSource.getLayout())); List probeJoinChannels = ImmutableList.copyOf(getChannelsForSymbols(probeSymbols, probeSource.getLayout())); OptionalInt probeHashChannel = probeHashSymbol.map(channelGetter(probeSource)) .map(OptionalInt::of).orElse(OptionalInt.empty()); - OptionalInt totalOperatorsCount = context.getDriverInstanceCount(); - checkState(!spillEnabled || totalOperatorsCount.isPresent(), "A fixed distribution is required for JOIN when spilling is enabled"); + OptionalInt totalOperatorsCount = OptionalInt.empty(); + if (spillEnabled) { + totalOperatorsCount = context.getDriverInstanceCount(); + checkState(totalOperatorsCount.isPresent(), "A fixed distribution is required for JOIN when spilling is enabled"); + } // Implementation of hash join operator may only take advantage of output duplicates insensitive joins when: // 1. Join is of INNER or LEFT type. For right or full joins all matching build rows must be tagged as visited. @@ -2335,6 +2352,9 @@ private OperatorFactory createLookupJoin( .map(JoinNode.EquiJoinClause::getRight) .collect(toImmutableSet()) .containsAll(node.getRightOutputSymbols()); + // Wait for build side to be collected before local dynamic filters are + // consumed by table scan. This way table scan can filter data more efficiently. + boolean waitForBuild = consumedLocalDynamicFilters; switch (node.getType()) { case INNER: return lookupJoinOperators.innerJoin( @@ -2343,6 +2363,7 @@ private OperatorFactory createLookupJoin( lookupSourceFactoryManager, probeTypes, outputSingleMatch, + waitForBuild, probeJoinChannels, probeHashChannel, Optional.of(probeOutputChannels), @@ -2363,7 +2384,18 @@ private OperatorFactory createLookupJoin( partitioningSpillerFactory, blockTypeOperators); case RIGHT: - return lookupJoinOperators.lookupOuterJoin(context.getNextOperatorId(), node.getId(), lookupSourceFactoryManager, probeTypes, probeJoinChannels, probeHashChannel, Optional.of(probeOutputChannels), totalOperatorsCount, partitioningSpillerFactory, blockTypeOperators); + return lookupJoinOperators.lookupOuterJoin( + context.getNextOperatorId(), + node.getId(), + lookupSourceFactoryManager, + probeTypes, + waitForBuild, + probeJoinChannels, + probeHashChannel, + Optional.of(probeOutputChannels), + totalOperatorsCount, + partitioningSpillerFactory, + blockTypeOperators); case FULL: return lookupJoinOperators.fullOuterJoin(context.getNextOperatorId(), node.getId(), lookupSourceFactoryManager, probeTypes, probeJoinChannels, probeHashChannel, Optional.of(probeOutputChannels), totalOperatorsCount, partitioningSpillerFactory, blockTypeOperators); } diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/OrderingScheme.java b/core/trino-main/src/main/java/io/trino/sql/planner/OrderingScheme.java index 9c2d514d11cf..61166b90befc 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/OrderingScheme.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/OrderingScheme.java @@ -18,7 +18,9 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import io.trino.spi.connector.LocalProperty; import io.trino.spi.connector.SortOrder; +import io.trino.spi.connector.SortingProperty; import io.trino.sql.tree.OrderBy; import io.trino.sql.tree.SortItem; import io.trino.sql.tree.SortItem.NullOrdering; @@ -138,4 +140,11 @@ public List toSortItems() io.trino.spi.connector.SortOrder.valueOf(getOrdering(symbol).name()))) .collect(toImmutableList()); } + + public List> toLocalProperties() + { + return getOrderBy().stream() + .map(symbol -> new SortingProperty<>(symbol, getOrdering(symbol))) + .collect(toImmutableList()); + } } diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/PlanOptimizers.java b/core/trino-main/src/main/java/io/trino/sql/planner/PlanOptimizers.java index c62db93d422b..d09c21b4e767 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/PlanOptimizers.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/PlanOptimizers.java @@ -580,23 +580,23 @@ public PlanOptimizers( new PruneTableScanColumns(metadata), new PushPredicateIntoTableScan(metadata, typeOperators, typeAnalyzer)))); + Set> pushIntoTableScanRulesExceptJoins = ImmutableSet.>builder() + .addAll(columnPruningRules) + .addAll(projectionPushdownRules) + .add(new PushProjectionIntoTableScan(metadata, typeAnalyzer)) + .add(new RemoveRedundantIdentityProjections()) + .add(new PushLimitIntoTableScan(metadata)) + .add(new PushPredicateIntoTableScan(metadata, typeOperators, typeAnalyzer)) + .add(new PushSampleIntoTableScan(metadata)) + .add(new PushAggregationIntoTableScan(metadata)) + .add(new PushDistinctLimitIntoTableScan(metadata)) + .add(new PushTopNIntoTableScan(metadata)) + .build(); IterativeOptimizer pushIntoTableScanOptimizer = new IterativeOptimizer( ruleStats, statsCalculator, estimatedExchangesCostCalculator, - ImmutableSet.>builder() - .addAll(columnPruningRules) - .addAll(projectionPushdownRules) - .add(new PushProjectionIntoTableScan(metadata, typeAnalyzer)) - .add(new RemoveRedundantIdentityProjections()) - .add(new PushLimitIntoTableScan(metadata)) - .add(new PushPredicateIntoTableScan(metadata, typeOperators, typeAnalyzer)) - .add(new PushSampleIntoTableScan(metadata)) - .add(new PushJoinIntoTableScan(metadata)) - .add(new PushAggregationIntoTableScan(metadata)) - .add(new PushDistinctLimitIntoTableScan(metadata)) - .add(new PushTopNIntoTableScan(metadata)) - .build()); + pushIntoTableScanRulesExceptJoins); builder.add(pushIntoTableScanOptimizer); builder.add(new UnaliasSymbolReferences(metadata)); builder.add(pushIntoTableScanOptimizer); // TODO (https://github.com/trinodb/trino/issues/811) merge with the above after migrating UnaliasSymbolReferences to rules @@ -734,7 +734,8 @@ public PlanOptimizers( new CreatePartialTopN(), new PushTopNThroughProject(), new PushTopNThroughOuterJoin(), - new PushTopNThroughUnion()))); + new PushTopNThroughUnion(), + new PushTopNIntoTableScan(metadata)))); builder.add(new IterativeOptimizer( ruleStats, statsCalculator, @@ -768,6 +769,21 @@ public PlanOptimizers( // Must run before AddExchanges and after ReplicateSemiJoinInDelete // to avoid temporarily having an invalid plan new DetermineSemiJoinDistributionType(costComparator, taskCountEstimator))))); + + builder.add(new IterativeOptimizer( + ruleStats, + statsCalculator, + estimatedExchangesCostCalculator, + ImmutableSet.>builder() + .addAll(pushIntoTableScanRulesExceptJoins) + // PushJoinIntoTableScan must run after ReorderJoins (and DetermineJoinDistributionType) + // otherwise too early pushdown could prevent optimal plan from being selected. + .add(new PushJoinIntoTableScan(metadata)) + // DetermineTableScanNodePartitioning is needed to needs to ensure all table handles have proper partitioning determined + // Must run before AddExchanges + .add(new DetermineTableScanNodePartitioning(metadata, nodePartitioningManager, taskCountEstimator)) + .build())); + builder.add( new IterativeOptimizer( ruleStats, diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PushTopNIntoTableScan.java b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PushTopNIntoTableScan.java index 96fe58c40231..fd11af8e371c 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PushTopNIntoTableScan.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PushTopNIntoTableScan.java @@ -25,7 +25,6 @@ import io.trino.sql.planner.plan.PlanNode; import io.trino.sql.planner.plan.TableScanNode; import io.trino.sql.planner.plan.TopNNode; -import io.trino.sql.planner.plan.TopNNode.Step; import java.util.List; import java.util.Map; @@ -37,16 +36,18 @@ import static io.trino.sql.planner.plan.Patterns.source; import static io.trino.sql.planner.plan.Patterns.tableScan; import static io.trino.sql.planner.plan.Patterns.topN; +import static io.trino.sql.planner.plan.TopNNode.Step.PARTIAL; +import static io.trino.sql.planner.plan.TopNNode.Step.SINGLE; public class PushTopNIntoTableScan implements Rule { private static final Capture TABLE_SCAN = newCapture(); - // Currently the rule is applied at the optimization phase where PARTIAL and FINAL TopNNode do not exist. - // The rule can be further made to work with PARTIAL and FINAL if needed. + // Rule is executed in two planning phases. Initially we try to pushdown SINGLE TopN into + // table scan. If that fails, we repeat the exercise for PARTIAL TopN nodes after SINGLE -> PARTIAL/FINAL split. private static final Pattern PATTERN = topN() - .matching(node -> node.getStep().equals(Step.SINGLE)) + .matching(node -> node.getStep() == SINGLE || node.getStep() == PARTIAL) .with(source().matching(tableScan().capturedAs(TABLE_SCAN))); private final Metadata metadata; @@ -91,6 +92,18 @@ public Result apply(TopNNode topNNode, Captures captures, Context context) // table scan partitioning might have changed with new table handle Optional.empty()); + // If possible we are getting rid of TopN node. + // + // If we are operating in `SINGLE` step and connector + // TopN pushdown is guaranteed we are removing TopN node from plan altogether. + + // For PARTIAL step it would be semantically correct to always drop TopN node from the plan, no matter if connector + // declares pushdown as guaranteed or not. But we decided to leave it in the plan for non-guaranteed pushdown, as there is no way + // to determine the size of output returned by connector. If connector pushdown support is very limited, and still a lot of data is returned + // after pushdown, removing PARTIAL TopN node would make query execution significantly more expensive. + // + // FINAL step of TopN node is never removed as it is needed to perform final filter higher in the query execution. + if (!result.isTopNGuaranteed()) { node = topNNode.replaceChildren(ImmutableList.of(node)); } diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedInPredicateToJoin.java b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedInPredicateToJoin.java index 1f3641c3cd7e..8467237e625f 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedInPredicateToJoin.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedInPredicateToJoin.java @@ -81,9 +81,9 @@ * Into: *
  * - Project (output: CASE WHEN (countmatches > 0) THEN true WHEN (countnullmatches > 0) THEN null ELSE false END)
- *   - Aggregate (countmatches=count(*) where a, b not null; countnullmatches where a,b null but buildSideKnownNonNull is not null)
+ *   - Aggregate (countmatches=count(*) where a, b not null; countnullmatches where (a is null or b is null) but buildSideKnownNonNull is not null)
  *     grouping by (A'.*)
- *     - LeftJoin on (A and B correlation condition)
+ *     - LeftJoin on (a = B.b, A and B correlation condition)
  *       - AssignUniqueId (A')
  *         - A
  * 
diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/AddExchanges.java b/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/AddExchanges.java index 4842bde4544c..4e194edfee30 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/AddExchanges.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/AddExchanges.java @@ -25,7 +25,6 @@ import io.trino.metadata.Metadata; import io.trino.spi.connector.GroupingProperty; import io.trino.spi.connector.LocalProperty; -import io.trino.spi.connector.SortingProperty; import io.trino.spi.type.TypeOperators; import io.trino.sql.planner.DomainTranslator; import io.trino.sql.planner.Partitioning; @@ -293,10 +292,7 @@ public PlanWithProperties visitWindow(WindowNode node, PreferredProperties prefe if (!node.getPartitionBy().isEmpty()) { desiredProperties.add(new GroupingProperty<>(node.getPartitionBy())); } - node.getOrderingScheme().ifPresent(orderingScheme -> - orderingScheme.getOrderBy().stream() - .map(symbol -> new SortingProperty<>(symbol, orderingScheme.getOrdering(symbol))) - .forEach(desiredProperties::add)); + node.getOrderingScheme().ifPresent(orderingScheme -> desiredProperties.addAll(orderingScheme.toLocalProperties())); PlanWithProperties child = planChild( node, @@ -428,10 +424,7 @@ public PlanWithProperties visitSort(SortNode node, PreferredProperties preferred // current plan so far is single node, so local properties are effectively global properties // skip the SortNode if the local properties guarantee ordering on Sort keys // TODO: This should be extracted as a separate optimizer once the planner is able to reason about the ordering of each operator - List> desiredProperties = new ArrayList<>(); - for (Symbol symbol : node.getOrderingScheme().getOrderBy()) { - desiredProperties.add(new SortingProperty<>(symbol, node.getOrderingScheme().getOrdering(symbol))); - } + List> desiredProperties = node.getOrderingScheme().toLocalProperties(); if (LocalProperties.match(child.getProperties().getLocalProperties(), desiredProperties).stream() .noneMatch(Optional::isPresent)) { diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/AddLocalExchanges.java b/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/AddLocalExchanges.java index c4f270836904..c92e1e1dcee2 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/AddLocalExchanges.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/AddLocalExchanges.java @@ -22,7 +22,6 @@ import io.trino.spi.connector.ConstantProperty; import io.trino.spi.connector.GroupingProperty; import io.trino.spi.connector.LocalProperty; -import io.trino.spi.connector.SortingProperty; import io.trino.spi.type.TypeOperators; import io.trino.sql.planner.Partitioning; import io.trino.sql.planner.PartitioningScheme; @@ -393,10 +392,7 @@ public PlanWithProperties visitWindow(WindowNode node, StreamPreferredProperties if (!node.getPartitionBy().isEmpty()) { desiredProperties.add(new GroupingProperty<>(node.getPartitionBy())); } - node.getOrderingScheme().ifPresent(orderingScheme -> - orderingScheme.getOrderBy().stream() - .map(symbol -> new SortingProperty<>(symbol, orderingScheme.getOrdering(symbol))) - .forEach(desiredProperties::add)); + node.getOrderingScheme().ifPresent(orderingScheme -> desiredProperties.addAll(orderingScheme.toLocalProperties())); Iterator>> matchIterator = LocalProperties.match(child.getProperties().getLocalProperties(), desiredProperties).iterator(); Set prePartitionedInputs = ImmutableSet.of(); diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/PropertyDerivations.java b/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/PropertyDerivations.java index 5e5de457db80..e7b8b2004383 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/PropertyDerivations.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/PropertyDerivations.java @@ -28,7 +28,6 @@ import io.trino.spi.connector.ConstantProperty; import io.trino.spi.connector.GroupingProperty; import io.trino.spi.connector.LocalProperty; -import io.trino.spi.connector.SortingProperty; import io.trino.spi.predicate.NullableValue; import io.trino.spi.type.Type; import io.trino.spi.type.TypeOperators; @@ -278,10 +277,7 @@ public ActualProperties visitWindow(WindowNode node, List inpu localProperties.add(new GroupingProperty<>(node.getPartitionBy())); } - orderingScheme.ifPresent(scheme -> - scheme.getOrderBy().stream() - .map(column -> new SortingProperty<>(column, scheme.getOrdering(column))) - .forEach(localProperties::add)); + orderingScheme.ifPresent(ordering -> localProperties.addAll(ordering.toLocalProperties())); return ActualProperties.builderFrom(properties) .local(LocalProperties.normalizeAndPrune(localProperties.build())) @@ -334,9 +330,7 @@ public ActualProperties visitTopNRanking(TopNRankingNode node, List> localProperties = ImmutableList.builder(); localProperties.add(new GroupingProperty<>(node.getPartitionBy())); - for (Symbol column : node.getOrderingScheme().getOrderBy()) { - localProperties.add(new SortingProperty<>(column, node.getOrderingScheme().getOrdering(column))); - } + localProperties.addAll(node.getOrderingScheme().toLocalProperties()); return ActualProperties.builderFrom(properties) .local(localProperties.build()) @@ -348,12 +342,8 @@ public ActualProperties visitTopN(TopNNode node, List inputPro { ActualProperties properties = Iterables.getOnlyElement(inputProperties); - List> localProperties = node.getOrderingScheme().getOrderBy().stream() - .map(column -> new SortingProperty<>(column, node.getOrderingScheme().getOrdering(column))) - .collect(toImmutableList()); - return ActualProperties.builderFrom(properties) - .local(localProperties) + .local(node.getOrderingScheme().toLocalProperties()) .build(); } @@ -362,12 +352,8 @@ public ActualProperties visitSort(SortNode node, List inputPro { ActualProperties properties = Iterables.getOnlyElement(inputProperties); - List> localProperties = node.getOrderingScheme().getOrderBy().stream() - .map(column -> new SortingProperty<>(column, node.getOrderingScheme().getOrdering(column))) - .collect(toImmutableList()); - return ActualProperties.builderFrom(properties) - .local(localProperties) + .local(node.getOrderingScheme().toLocalProperties()) .build(); } @@ -565,12 +551,8 @@ public ActualProperties visitExchange(ExchangeNode node, List Map constants = entries.stream() .collect(toMap(Map.Entry::getKey, Map.Entry::getValue)); - ImmutableList.Builder> localProperties = ImmutableList.builder(); - if (node.getOrderingScheme().isPresent()) { - node.getOrderingScheme().get().getOrderBy().stream() - .map(column -> new SortingProperty<>(column, node.getOrderingScheme().get().getOrdering(column))) - .forEach(localProperties::add); - } + ImmutableList.Builder> localProperties = ImmutableList.builder(); + node.getOrderingScheme().ifPresent(orderingScheme -> localProperties.addAll(orderingScheme.toLocalProperties())); // Local exchanges are only created in AddLocalExchanges, at the end of optimization, and // local exchanges do not produce all global properties as represented by ActualProperties. diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/planprinter/TableInfoSupplier.java b/core/trino-main/src/main/java/io/trino/sql/planner/planprinter/TableInfoSupplier.java index 4d7b97a54841..5558b03e7c85 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/planprinter/TableInfoSupplier.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/planprinter/TableInfoSupplier.java @@ -16,8 +16,8 @@ import io.trino.Session; import io.trino.execution.TableInfo; import io.trino.metadata.Metadata; -import io.trino.metadata.TableMetadata; import io.trino.metadata.TableProperties; +import io.trino.metadata.TableSchema; import io.trino.sql.planner.plan.TableScanNode; import java.util.function.Function; @@ -39,8 +39,8 @@ public TableInfoSupplier(Metadata metadata, Session session) @Override public TableInfo apply(TableScanNode node) { - TableMetadata tableMetadata = metadata.getTableMetadata(session, node.getTable()); + TableSchema tableSchema = metadata.getTableSchema(session, node.getTable()); TableProperties tableProperties = metadata.getTableProperties(session, node.getTable()); - return new TableInfo(tableMetadata.getQualifiedName(), tableProperties.getPredicate()); + return new TableInfo(tableSchema.getQualifiedName(), tableProperties.getPredicate()); } } diff --git a/core/trino-main/src/main/java/io/trino/sql/rewrite/ShowQueriesRewrite.java b/core/trino-main/src/main/java/io/trino/sql/rewrite/ShowQueriesRewrite.java index 720f9db5c6ac..3ca6ef720cf1 100644 --- a/core/trino-main/src/main/java/io/trino/sql/rewrite/ShowQueriesRewrite.java +++ b/core/trino-main/src/main/java/io/trino/sql/rewrite/ShowQueriesRewrite.java @@ -474,20 +474,20 @@ private static Expression toExpression(Object value) @Override protected Node visitShowCreate(ShowCreate node, Void context) { - if (node.getType() == VIEW) { + if (node.getType() == MATERIALIZED_VIEW) { QualifiedObjectName objectName = createQualifiedObjectName(session, node, node.getName()); - - if (metadata.getMaterializedView(session, objectName).isPresent()) { - throw semanticException(NOT_SUPPORTED, node, "Relation '%s' is a materialized view, not a view", objectName); - } - - Optional viewDefinition = metadata.getView(session, objectName); + Optional viewDefinition = metadata.getMaterializedView(session, objectName); if (viewDefinition.isEmpty()) { + if (metadata.getView(session, objectName).isPresent()) { + throw semanticException(NOT_SUPPORTED, node, "Relation '%s' is a view, not a materialized view", objectName); + } + if (metadata.getTableHandle(session, objectName).isPresent()) { - throw semanticException(NOT_SUPPORTED, node, "Relation '%s' is a table, not a view", objectName); + throw semanticException(NOT_SUPPORTED, node, "Relation '%s' is a table, not a materialized view", objectName); } - throw semanticException(TABLE_NOT_FOUND, node, "View '%s' does not exist", objectName); + + throw semanticException(TABLE_NOT_FOUND, node, "Materialized view '%s' does not exist", objectName); } Query query = parseView(viewDefinition.get().getOriginalSql(), objectName, node); @@ -498,18 +498,23 @@ protected Node visitShowCreate(ShowCreate node, Void context) accessControl.checkCanShowCreateTable(session.toSecurityContext(), new QualifiedObjectName(catalogName.getValue(), schemaName.getValue(), tableName.getValue())); - CreateView.Security security = viewDefinition.get().isRunAsInvoker() ? INVOKER : DEFINER; - String sql = formatSql(new CreateView(QualifiedName.of(ImmutableList.of(catalogName, schemaName, tableName)), query, false, viewDefinition.get().getComment(), Optional.of(security))).trim(); - return singleValueQuery("Create View", sql); + String sql = formatSql(new CreateMaterializedView(Optional.empty(), QualifiedName.of(ImmutableList.of(catalogName, schemaName, tableName)), + query, false, false, new ArrayList<>(), viewDefinition.get().getComment())).trim(); + return singleValueQuery("Create Materialized View", sql); } - if (node.getType() == MATERIALIZED_VIEW) { + if (node.getType() == VIEW) { QualifiedObjectName objectName = createQualifiedObjectName(session, node, node.getName()); - Optional viewDefinition = metadata.getMaterializedView(session, objectName); + + if (metadata.getMaterializedView(session, objectName).isPresent()) { + throw semanticException(NOT_SUPPORTED, node, "Relation '%s' is a materialized view, not a view", objectName); + } + + Optional viewDefinition = metadata.getView(session, objectName); if (viewDefinition.isEmpty()) { if (metadata.getTableHandle(session, objectName).isPresent()) { - throw semanticException(NOT_SUPPORTED, node, "Relation '%s' is a table, not a materialized view", objectName); + throw semanticException(NOT_SUPPORTED, node, "Relation '%s' is a table, not a view", objectName); } throw semanticException(TABLE_NOT_FOUND, node, "View '%s' does not exist", objectName); } @@ -522,16 +527,19 @@ protected Node visitShowCreate(ShowCreate node, Void context) accessControl.checkCanShowCreateTable(session.toSecurityContext(), new QualifiedObjectName(catalogName.getValue(), schemaName.getValue(), tableName.getValue())); - String sql = formatSql(new CreateMaterializedView(Optional.empty(), QualifiedName.of(ImmutableList.of(catalogName, schemaName, tableName)), - query, false, false, new ArrayList<>(), viewDefinition.get().getComment())).trim(); - return singleValueQuery("Create Materialized View", sql); + CreateView.Security security = viewDefinition.get().isRunAsInvoker() ? INVOKER : DEFINER; + String sql = formatSql(new CreateView(QualifiedName.of(ImmutableList.of(catalogName, schemaName, tableName)), query, false, viewDefinition.get().getComment(), Optional.of(security))).trim(); + return singleValueQuery("Create View", sql); } if (node.getType() == TABLE) { QualifiedObjectName objectName = createQualifiedObjectName(session, node, node.getName()); - Optional viewDefinition = metadata.getView(session, objectName); - if (viewDefinition.isPresent()) { + if (metadata.getMaterializedView(session, objectName).isPresent()) { + throw semanticException(NOT_SUPPORTED, node, "Relation '%s' is a materialized view, not a table", objectName); + } + + if (metadata.getView(session, objectName).isPresent()) { throw semanticException(NOT_SUPPORTED, node, "Relation '%s' is a view, not a table", objectName); } diff --git a/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java b/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java index 1e9acf00db7f..848e31b9ba24 100644 --- a/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java +++ b/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java @@ -108,6 +108,7 @@ import io.trino.server.PluginManager; import io.trino.server.SessionPropertyDefaults; import io.trino.server.security.CertificateAuthenticatorManager; +import io.trino.server.security.PasswordAuthenticatorConfig; import io.trino.server.security.PasswordAuthenticatorManager; import io.trino.spi.PageIndexerFactory; import io.trino.spi.PageSorter; @@ -381,7 +382,7 @@ private LocalQueryRunner( metadata, new NoOpResourceGroupManager(), accessControl, - new PasswordAuthenticatorManager(), + Optional.of(new PasswordAuthenticatorManager(new PasswordAuthenticatorConfig())), new CertificateAuthenticatorManager(), eventListenerManager, new GroupProviderManager(), diff --git a/core/trino-main/src/main/java/io/trino/testing/TestingMetadata.java b/core/trino-main/src/main/java/io/trino/testing/TestingMetadata.java index 8653db038c6f..2be2b942029a 100644 --- a/core/trino-main/src/main/java/io/trino/testing/TestingMetadata.java +++ b/core/trino-main/src/main/java/io/trino/testing/TestingMetadata.java @@ -23,6 +23,7 @@ import io.trino.spi.connector.ColumnHandle; import io.trino.spi.connector.ColumnMetadata; import io.trino.spi.connector.ConnectorInsertTableHandle; +import io.trino.spi.connector.ConnectorMaterializedViewDefinition; import io.trino.spi.connector.ConnectorMetadata; import io.trino.spi.connector.ConnectorNewTableLayout; import io.trino.spi.connector.ConnectorOutputMetadata; @@ -61,6 +62,7 @@ public class TestingMetadata { private final ConcurrentMap tables = new ConcurrentHashMap<>(); private final ConcurrentMap views = new ConcurrentHashMap<>(); + private final ConcurrentMap materializedViews = new ConcurrentHashMap<>(); @Override public List listSchemaNames(ConnectorSession session) @@ -218,6 +220,26 @@ public Optional getView(ConnectorSession session, Schem return Optional.ofNullable(views.get(viewName)); } + @Override + public void createMaterializedView(ConnectorSession session, SchemaTableName viewName, ConnectorMaterializedViewDefinition definition, boolean replace, boolean ignoreExisting) + { + if (replace) { + materializedViews.put(viewName, definition); + } + else if (materializedViews.putIfAbsent(viewName, definition) != null) { + if (ignoreExisting) { + return; + } + throw new TrinoException(ALREADY_EXISTS, "Materialized view already exists: " + viewName); + } + } + + @Override + public Optional getMaterializedView(ConnectorSession session, SchemaTableName viewName) + { + return Optional.ofNullable(materializedViews.get(viewName)); + } + @Override public ConnectorOutputTableHandle beginCreateTable(ConnectorSession session, ConnectorTableMetadata tableMetadata, Optional layout) { diff --git a/core/trino-main/src/test/java/io/trino/connector/MockConnector.java b/core/trino-main/src/test/java/io/trino/connector/MockConnector.java index 00e387ddb476..784cd22c3a8b 100644 --- a/core/trino-main/src/test/java/io/trino/connector/MockConnector.java +++ b/core/trino-main/src/test/java/io/trino/connector/MockConnector.java @@ -14,6 +14,8 @@ package io.trino.connector; import com.google.common.collect.ImmutableList; +import io.airlift.slice.Slice; +import io.trino.spi.Page; import io.trino.spi.connector.AggregateFunction; import io.trino.spi.connector.AggregationApplicationResult; import io.trino.spi.connector.ColumnHandle; @@ -24,8 +26,14 @@ import io.trino.spi.connector.ConnectorMaterializedViewDefinition; import io.trino.spi.connector.ConnectorMetadata; import io.trino.spi.connector.ConnectorNewTableLayout; +import io.trino.spi.connector.ConnectorOutputMetadata; import io.trino.spi.connector.ConnectorOutputTableHandle; +import io.trino.spi.connector.ConnectorPageSink; +import io.trino.spi.connector.ConnectorPageSinkProvider; +import io.trino.spi.connector.ConnectorPageSource; +import io.trino.spi.connector.ConnectorPageSourceProvider; import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.ConnectorSplit; import io.trino.spi.connector.ConnectorSplitManager; import io.trino.spi.connector.ConnectorSplitSource; import io.trino.spi.connector.ConnectorTableHandle; @@ -36,6 +44,7 @@ import io.trino.spi.connector.Constraint; import io.trino.spi.connector.ConstraintApplicationResult; import io.trino.spi.connector.DynamicFilter; +import io.trino.spi.connector.EmptyPageSource; import io.trino.spi.connector.FixedSplitSource; import io.trino.spi.connector.JoinApplicationResult; import io.trino.spi.connector.JoinCondition; @@ -52,20 +61,25 @@ import io.trino.spi.security.Privilege; import io.trino.spi.security.RoleGrant; import io.trino.spi.security.TrinoPrincipal; +import io.trino.spi.statistics.ComputedStatistics; import io.trino.spi.transaction.IsolationLevel; +import java.util.Collection; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.OptionalLong; import java.util.Set; +import java.util.concurrent.CompletableFuture; import java.util.function.BiFunction; import java.util.function.Function; import java.util.function.Supplier; import static com.google.common.collect.ImmutableMap.toImmutableMap; import static com.google.common.collect.ImmutableSet.toImmutableSet; +import static io.trino.spi.type.BigintType.BIGINT; import static java.util.Objects.requireNonNull; +import static java.util.concurrent.CompletableFuture.completedFuture; public class MockConnector implements Connector @@ -73,6 +87,7 @@ public class MockConnector private final Function> listSchemaNames; private final BiFunction> listTables; private final BiFunction> getViews; + private final BiFunction> getMaterializedViews; private final BiFunction getTableHandle; private final Function> getColumns; private final MockConnectorFactory.ApplyProjection applyProjection; @@ -92,6 +107,7 @@ public class MockConnector Function> listSchemaNames, BiFunction> listTables, BiFunction> getViews, + BiFunction> getMaterializedViews, BiFunction getTableHandle, Function> getColumns, MockConnectorFactory.ApplyProjection applyProjection, @@ -110,6 +126,7 @@ public class MockConnector this.listSchemaNames = requireNonNull(listSchemaNames, "listSchemaNames is null"); this.listTables = requireNonNull(listTables, "listTables is null"); this.getViews = requireNonNull(getViews, "getViews is null"); + this.getMaterializedViews = requireNonNull(getMaterializedViews, "getMaterializedViews is null"); this.getTableHandle = requireNonNull(getTableHandle, "getTableHandle is null"); this.getColumns = requireNonNull(getColumns, "getColumns is null"); this.applyProjection = requireNonNull(applyProjection, "applyProjection is null"); @@ -138,6 +155,18 @@ public ConnectorMetadata getMetadata(ConnectorTransactionHandle transaction) return new MockConnectorMetadata(); } + @Override + public ConnectorPageSourceProvider getPageSourceProvider() + { + return new MockPageSourceProvider(); + } + + @Override + public ConnectorPageSinkProvider getPageSinkProvider() + { + return new MockPageSinkProvider(); + } + @Override public ConnectorSplitManager getSplitManager() { @@ -235,6 +264,15 @@ public List listSchemaNames(ConnectorSession session) return listSchemaNames.apply(session); } + @Override + public void createSchema(ConnectorSession session, String schemaName, Map properties, TrinoPrincipal owner) {} + + @Override + public void renameSchema(ConnectorSession session, String source, String target) {} + + @Override + public void dropSchema(ConnectorSession session, String schemaName) {} + @Override public ConnectorTableHandle getTableHandle(ConnectorSession session, SchemaTableName tableName) { @@ -285,12 +323,40 @@ public Map> listTableColumns(ConnectorSess } @Override - public void createView(ConnectorSession session, SchemaTableName viewName, ConnectorViewDefinition definition, boolean replace) - {} + public void createTable(ConnectorSession session, ConnectorTableMetadata tableMetadata, boolean ignoreExisting) {} + + @Override + public void createView(ConnectorSession session, SchemaTableName viewName, ConnectorViewDefinition definition, boolean replace) {} @Override - public void createMaterializedView(ConnectorSession session, SchemaTableName viewName, ConnectorMaterializedViewDefinition definition, boolean replace, boolean ignoreExisting) - {} + public void createMaterializedView(ConnectorSession session, SchemaTableName viewName, ConnectorMaterializedViewDefinition definition, boolean replace, boolean ignoreExisting) {} + + @Override + public Optional getMaterializedView(ConnectorSession session, SchemaTableName viewName) + { + return Optional.ofNullable(getMaterializedViews.apply(session, viewName.toSchemaTablePrefix()).get(viewName)); + } + + @Override + public ConnectorInsertTableHandle beginRefreshMaterializedView(ConnectorSession session, ConnectorTableHandle tableHandle, List sourceTableHandles) + { + return new MockConnectorInsertTableHandle(((MockConnectorTableHandle) tableHandle).getTableName()); + } + + @Override + public Optional finishRefreshMaterializedView( + ConnectorSession session, + ConnectorTableHandle tableHandle, + ConnectorInsertTableHandle insertHandle, + Collection fragments, + Collection computedStatistics, + List sourceTableHandles) + { + return Optional.empty(); + } + + @Override + public void dropMaterializedView(ConnectorSession session, SchemaTableName viewName) {} @Override public Map getViews(ConnectorSession session, Optional schemaName) @@ -305,9 +371,15 @@ public Optional getView(ConnectorSession session, Schem } @Override - public ConnectorInsertTableHandle beginInsert(ConnectorSession session, ConnectorTableHandle tableHandle) + public ConnectorInsertTableHandle beginInsert(ConnectorSession session, ConnectorTableHandle tableHandle, List columns) { - return new MockConnectorInsertTableHandle(); + return new MockConnectorInsertTableHandle(((MockConnectorTableHandle) tableHandle).getTableName()); + } + + @Override + public Optional finishInsert(ConnectorSession session, ConnectorInsertTableHandle insertHandle, Collection fragments, Collection computedStatistics) + { + return Optional.empty(); } @Override @@ -320,7 +392,13 @@ public Optional getInsertLayout(ConnectorSession sessio @Override public ConnectorOutputTableHandle beginCreateTable(ConnectorSession session, ConnectorTableMetadata tableMetadata, Optional layout) { - return new MockConnectorOutputTableHandle(); + return new MockConnectorOutputTableHandle(tableMetadata.getTable()); + } + + @Override + public Optional finishCreateTable(ConnectorSession session, ConnectorOutputTableHandle tableHandle, Collection fragments, Collection computedStatistics) + { + return Optional.empty(); } @Override @@ -329,6 +407,21 @@ public Optional getNewTableLayout(ConnectorSession sess return getNewTableLayout.apply(session, tableMetadata); } + @Override + public ConnectorTableHandle beginUpdate(ConnectorSession session, ConnectorTableHandle tableHandle, List updatedColumns) + { + return tableHandle; + } + + @Override + public void finishUpdate(ConnectorSession session, ConnectorTableHandle tableHandle, Collection fragments) {} + + @Override + public ColumnHandle getUpdateRowIdColumnHandle(ConnectorSession session, ConnectorTableHandle tableHandle, List updatedColumns) + { + return new MockConnectorColumnHandle("update_row_id", BIGINT); + } + @Override public boolean usesLegacyTableLayouts() { @@ -395,4 +488,56 @@ public void revokeTablePrivileges(ConnectorSession session, SchemaTableName tabl accessControl.revokeTablePrivileges(tableName, privileges, revokee, grantOption); } } + + private static class MockPageSinkProvider + implements ConnectorPageSinkProvider + { + @Override + public ConnectorPageSink createPageSink(ConnectorTransactionHandle transactionHandle, ConnectorSession session, ConnectorOutputTableHandle outputTableHandle) + { + return new MockPageSink(); + } + + @Override + public ConnectorPageSink createPageSink(ConnectorTransactionHandle transactionHandle, ConnectorSession session, ConnectorInsertTableHandle insertTableHandle) + { + return new MockPageSink(); + } + } + + private static class MockPageSink + implements ConnectorPageSink + { + @Override + public CompletableFuture appendPage(Page page) + { + return NOT_BLOCKED; + } + + @Override + public CompletableFuture> finish() + { + return completedFuture(ImmutableList.of()); + } + + @Override + public void abort() {} + } + + private static class MockPageSourceProvider + implements ConnectorPageSourceProvider + { + @Override + public ConnectorPageSource createPageSource(ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorSplit split, ConnectorTableHandle table, List columns, DynamicFilter dynamicFilter) + { + return new MockPageSource(); + } + } + + private static class MockPageSource + extends EmptyPageSource + { + @Override + public void updateRows(Page page, List columnValueAndRowIdChannels) {} + } } diff --git a/core/trino-main/src/test/java/io/trino/connector/MockConnectorFactory.java b/core/trino-main/src/test/java/io/trino/connector/MockConnectorFactory.java index c6222dd06052..be788f409e94 100644 --- a/core/trino-main/src/test/java/io/trino/connector/MockConnectorFactory.java +++ b/core/trino-main/src/test/java/io/trino/connector/MockConnectorFactory.java @@ -24,6 +24,7 @@ import io.trino.spi.connector.ConnectorContext; import io.trino.spi.connector.ConnectorFactory; import io.trino.spi.connector.ConnectorHandleResolver; +import io.trino.spi.connector.ConnectorMaterializedViewDefinition; import io.trino.spi.connector.ConnectorNewTableLayout; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.ConnectorTableHandle; @@ -66,6 +67,7 @@ public class MockConnectorFactory private final Function> listSchemaNames; private final BiFunction> listTables; private final BiFunction> getViews; + private final BiFunction> getMaterializedViews; private final BiFunction getTableHandle; private final Function> getColumns; private final ApplyProjection applyProjection; @@ -85,6 +87,7 @@ private MockConnectorFactory( Function> listSchemaNames, BiFunction> listTables, BiFunction> getViews, + BiFunction> getMaterializedViews, BiFunction getTableHandle, Function> getColumns, ApplyProjection applyProjection, @@ -103,6 +106,7 @@ private MockConnectorFactory( this.listSchemaNames = requireNonNull(listSchemaNames, "listSchemaNames is null"); this.listTables = requireNonNull(listTables, "listTables is null"); this.getViews = requireNonNull(getViews, "getViews is null"); + this.getMaterializedViews = requireNonNull(getMaterializedViews, "getMaterializedViews is null"); this.getTableHandle = requireNonNull(getTableHandle, "getTableHandle is null"); this.getColumns = requireNonNull(getColumns, "getColumns is null"); this.applyProjection = requireNonNull(applyProjection, "applyProjection is null"); @@ -138,6 +142,7 @@ public Connector create(String catalogName, Map config, Connecto listSchemaNames, listTables, getViews, + getMaterializedViews, getTableHandle, getColumns, applyProjection, @@ -227,6 +232,7 @@ public static final class Builder private Function> listSchemaNames = defaultListSchemaNames(); private BiFunction> listTables = defaultListTables(); private BiFunction> getViews = defaultGetViews(); + private BiFunction> getMaterializedViews = defaultGetMaterializedViews(); private BiFunction getTableHandle = defaultGetTableHandle(); private Function> getColumns = defaultGetColumns(); private ApplyProjection applyProjection = (session, handle, projections, assignments) -> Optional.empty(); @@ -269,6 +275,12 @@ public Builder withGetViews(BiFunction> getMaterializedViews) + { + this.getMaterializedViews = requireNonNull(getMaterializedViews, "getMaterializedViews is null"); + return this; + } + public Builder withGetTableHandle(BiFunction getTableHandle) { this.getTableHandle = requireNonNull(getTableHandle, "getTableHandle is null"); @@ -381,6 +393,7 @@ public MockConnectorFactory build() listSchemaNames, listTables, getViews, + getMaterializedViews, getTableHandle, getColumns, applyProjection, @@ -417,6 +430,11 @@ public static BiFunction ImmutableMap.of(); } + public static BiFunction> defaultGetMaterializedViews() + { + return (session, schemaTablePrefix) -> ImmutableMap.of(); + } + public static BiFunction defaultGetTableHandle() { return (session, schemaTableName) -> new MockConnectorTableHandle(schemaTableName); diff --git a/core/trino-main/src/test/java/io/trino/connector/MockConnectorHandleResolver.java b/core/trino-main/src/test/java/io/trino/connector/MockConnectorHandleResolver.java index 8c940a220130..e4d673e78eec 100644 --- a/core/trino-main/src/test/java/io/trino/connector/MockConnectorHandleResolver.java +++ b/core/trino-main/src/test/java/io/trino/connector/MockConnectorHandleResolver.java @@ -15,6 +15,8 @@ import io.trino.spi.connector.ColumnHandle; import io.trino.spi.connector.ConnectorHandleResolver; +import io.trino.spi.connector.ConnectorInsertTableHandle; +import io.trino.spi.connector.ConnectorOutputTableHandle; import io.trino.spi.connector.ConnectorTableHandle; import io.trino.spi.connector.ConnectorTransactionHandle; @@ -38,4 +40,16 @@ public Class getColumnHandleClass() { return MockConnectorColumnHandle.class; } + + @Override + public Class getOutputTableHandleClass() + { + return MockConnectorOutputTableHandle.class; + } + + @Override + public Class getInsertTableHandleClass() + { + return MockConnectorInsertTableHandle.class; + } } diff --git a/core/trino-main/src/test/java/io/trino/connector/MockConnectorInsertTableHandle.java b/core/trino-main/src/test/java/io/trino/connector/MockConnectorInsertTableHandle.java index 0884563b8479..341a8ec67fa9 100644 --- a/core/trino-main/src/test/java/io/trino/connector/MockConnectorInsertTableHandle.java +++ b/core/trino-main/src/test/java/io/trino/connector/MockConnectorInsertTableHandle.java @@ -13,9 +13,48 @@ */ package io.trino.connector; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import io.trino.spi.connector.ConnectorInsertTableHandle; +import io.trino.spi.connector.SchemaTableName; -class MockConnectorInsertTableHandle +import java.util.Objects; + +import static java.util.Objects.requireNonNull; + +public class MockConnectorInsertTableHandle implements ConnectorInsertTableHandle { + private final SchemaTableName tableName; + + @JsonCreator + public MockConnectorInsertTableHandle(@JsonProperty("tableName") SchemaTableName tableName) + { + this.tableName = requireNonNull(tableName, "tableName is null"); + } + + @JsonProperty + public SchemaTableName getTableName() + { + return tableName; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + MockConnectorInsertTableHandle other = (MockConnectorInsertTableHandle) o; + return Objects.equals(tableName, other.tableName); + } + + @Override + public int hashCode() + { + return Objects.hash(tableName); + } } diff --git a/core/trino-main/src/test/java/io/trino/connector/MockConnectorOutputTableHandle.java b/core/trino-main/src/test/java/io/trino/connector/MockConnectorOutputTableHandle.java index 015bca74a6f5..54d2e6758cf2 100644 --- a/core/trino-main/src/test/java/io/trino/connector/MockConnectorOutputTableHandle.java +++ b/core/trino-main/src/test/java/io/trino/connector/MockConnectorOutputTableHandle.java @@ -13,9 +13,48 @@ */ package io.trino.connector; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import io.trino.spi.connector.ConnectorOutputTableHandle; +import io.trino.spi.connector.SchemaTableName; -class MockConnectorOutputTableHandle +import java.util.Objects; + +import static java.util.Objects.requireNonNull; + +public class MockConnectorOutputTableHandle implements ConnectorOutputTableHandle { + private final SchemaTableName tableName; + + @JsonCreator + public MockConnectorOutputTableHandle(@JsonProperty("tableHandle") SchemaTableName tableName) + { + this.tableName = requireNonNull(tableName, "tableName is null"); + } + + @JsonProperty + public SchemaTableName getTableName() + { + return tableName; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + MockConnectorOutputTableHandle other = (MockConnectorOutputTableHandle) o; + return Objects.equals(tableName, other.tableName); + } + + @Override + public int hashCode() + { + return Objects.hash(tableName); + } } diff --git a/core/trino-main/src/test/java/io/trino/execution/TestCreateTableTask.java b/core/trino-main/src/test/java/io/trino/execution/TestCreateTableTask.java index 86f6f7111817..cfb9f95f8c66 100644 --- a/core/trino-main/src/test/java/io/trino/execution/TestCreateTableTask.java +++ b/core/trino-main/src/test/java/io/trino/execution/TestCreateTableTask.java @@ -134,7 +134,7 @@ public void testCreateTableNotExistsTrue() ImmutableList.of(), Optional.empty()); - getFutureValue(new CreateTableTask().internalExecute(statement, metadata, new AllowAllAccessControl(), testSession, emptyList())); + getFutureValue(new CreateTableTask().internalExecute(statement, metadata, new AllowAllAccessControl(), testSession, emptyList(), output -> {})); assertEquals(metadata.getCreateTableCallCount(), 1); } @@ -147,7 +147,7 @@ public void testCreateTableNotExistsFalse() ImmutableList.of(), Optional.empty()); - assertTrinoExceptionThrownBy(() -> getFutureValue(new CreateTableTask().internalExecute(statement, metadata, new AllowAllAccessControl(), testSession, emptyList()))) + assertTrinoExceptionThrownBy(() -> getFutureValue(new CreateTableTask().internalExecute(statement, metadata, new AllowAllAccessControl(), testSession, emptyList(), output -> {}))) .hasErrorCode(ALREADY_EXISTS) .hasMessage("Table already exists"); @@ -164,7 +164,7 @@ public void testCreateWithNotNullColumns() new ColumnDefinition(identifier("c"), toSqlType(VARBINARY), false, emptyList(), Optional.empty())); CreateTable statement = new CreateTable(QualifiedName.of("test_table"), inputColumns, true, ImmutableList.of(), Optional.empty()); - getFutureValue(new CreateTableTask().internalExecute(statement, metadata, new AllowAllAccessControl(), testSession, emptyList())); + getFutureValue(new CreateTableTask().internalExecute(statement, metadata, new AllowAllAccessControl(), testSession, emptyList(), output -> {})); assertEquals(metadata.getCreateTableCallCount(), 1); List columns = metadata.getReceivedTableMetadata().get(0).getColumns(); assertEquals(columns.size(), 3); @@ -197,7 +197,7 @@ public void testCreateWithUnsupportedConnectorThrowsWhenNotNull() Optional.empty()); assertTrinoExceptionThrownBy(() -> - getFutureValue(new CreateTableTask().internalExecute(statement, metadata, new AllowAllAccessControl(), testSession, emptyList()))) + getFutureValue(new CreateTableTask().internalExecute(statement, metadata, new AllowAllAccessControl(), testSession, emptyList(), output -> {}))) .hasErrorCode(NOT_SUPPORTED) .hasMessage("Catalog 'catalog' does not support non-null column for column name 'b'"); } @@ -207,7 +207,7 @@ public void testCreateLike() { CreateTable statement = getCreatleLikeStatement(false); - getFutureValue(new CreateTableTask().internalExecute(statement, metadata, new AllowAllAccessControl(), testSession, List.of())); + getFutureValue(new CreateTableTask().internalExecute(statement, metadata, new AllowAllAccessControl(), testSession, List.of(), output -> {})); assertEquals(metadata.getCreateTableCallCount(), 1); assertThat(metadata.getReceivedTableMetadata().get(0).getColumns()) @@ -220,7 +220,7 @@ public void testCreateLikeWithProperties() { CreateTable statement = getCreatleLikeStatement(true); - getFutureValue(new CreateTableTask().internalExecute(statement, metadata, new AllowAllAccessControl(), testSession, List.of())); + getFutureValue(new CreateTableTask().internalExecute(statement, metadata, new AllowAllAccessControl(), testSession, List.of(), output -> {})); assertEquals(metadata.getCreateTableCallCount(), 1); assertThat(metadata.getReceivedTableMetadata().get(0).getColumns()) @@ -237,7 +237,7 @@ public void testCreateLikeDenyPermission() TestingAccessControlManager accessControl = new TestingAccessControlManager(transactionManager, new EventListenerManager(new EventListenerConfig())); accessControl.deny(privilege("parent_table", SELECT_COLUMN)); - assertThatThrownBy(() -> getFutureValue(new CreateTableTask().internalExecute(statement, metadata, accessControl, testSession, List.of()))) + assertThatThrownBy(() -> getFutureValue(new CreateTableTask().internalExecute(statement, metadata, accessControl, testSession, List.of(), output -> {}))) .isInstanceOf(AccessDeniedException.class) .hasMessageContaining("Cannot reference columns of table"); } @@ -250,7 +250,7 @@ public void testCreateLikeWithPropertiesDenyPermission() TestingAccessControlManager accessControl = new TestingAccessControlManager(transactionManager, new EventListenerManager(new EventListenerConfig())); accessControl.deny(privilege("parent_table", SHOW_CREATE_TABLE)); - assertThatThrownBy(() -> getFutureValue(new CreateTableTask().internalExecute(statement, metadata, accessControl, testSession, List.of()))) + assertThatThrownBy(() -> getFutureValue(new CreateTableTask().internalExecute(statement, metadata, accessControl, testSession, List.of(), output -> {}))) .isInstanceOf(AccessDeniedException.class) .hasMessageContaining("Cannot reference properties of table"); } diff --git a/core/trino-main/src/test/java/io/trino/execution/scheduler/TestFileBasedNetworkTopology.java b/core/trino-main/src/test/java/io/trino/execution/scheduler/TestFileBasedNetworkTopology.java index bb1b388f4258..d656f1f07d20 100644 --- a/core/trino-main/src/test/java/io/trino/execution/scheduler/TestFileBasedNetworkTopology.java +++ b/core/trino-main/src/test/java/io/trino/execution/scheduler/TestFileBasedNetworkTopology.java @@ -25,6 +25,7 @@ import java.io.File; import java.util.concurrent.TimeUnit; +import static java.util.concurrent.TimeUnit.DAYS; import static org.testng.Assert.assertEquals; public class TestFileBasedNetworkTopology @@ -43,7 +44,7 @@ public void setup() @Test public void testLocate() { - NetworkTopology topology = new FileBasedNetworkTopology(topologyFile, Duration.valueOf("1d"), new TestingTicker()); + NetworkTopology topology = new FileBasedNetworkTopology(topologyFile, new Duration(1, DAYS), new TestingTicker()); assertEquals(topology.locate(HostAddress.fromString("0.0.0.0")), new NetworkLocation()); assertEquals(topology.locate(HostAddress.fromString("not-exist.example.com")), new NetworkLocation()); @@ -63,7 +64,7 @@ public void testRefresh() Files.copy(topologyFile, tempFile.file()); TestingTicker ticker = new TestingTicker(); - FileBasedNetworkTopology topology = new FileBasedNetworkTopology(tempFile.file(), Duration.valueOf("1d"), ticker); + FileBasedNetworkTopology topology = new FileBasedNetworkTopology(tempFile.file(), new Duration(1, DAYS), ticker); assertEquals(topology.locate(HostAddress.fromString("not-exist.example.com")), new NetworkLocation()); assertEquals(topology.locate(HostAddress.fromString("192.168.0.1")), new NetworkLocation("region1", "rack1", "machine1")); diff --git a/core/trino-main/src/test/java/io/trino/metadata/AbstractMockMetadata.java b/core/trino-main/src/test/java/io/trino/metadata/AbstractMockMetadata.java index 8700a21623ef..d1df73ff6e36 100644 --- a/core/trino-main/src/test/java/io/trino/metadata/AbstractMockMetadata.java +++ b/core/trino-main/src/test/java/io/trino/metadata/AbstractMockMetadata.java @@ -163,6 +163,12 @@ public Optional getInfo(Session session, TableHandle handle) throw new UnsupportedOperationException(); } + @Override + public TableSchema getTableSchema(Session session, TableHandle tableHandle) + { + throw new UnsupportedOperationException(); + } + @Override public TableMetadata getTableMetadata(Session session, TableHandle tableHandle) { diff --git a/core/trino-main/src/test/java/io/trino/operator/BenchmarkHashBuildAndJoinOperators.java b/core/trino-main/src/test/java/io/trino/operator/BenchmarkHashBuildAndJoinOperators.java index dc46034b6ae1..ac0fa350bddd 100644 --- a/core/trino-main/src/test/java/io/trino/operator/BenchmarkHashBuildAndJoinOperators.java +++ b/core/trino-main/src/test/java/io/trino/operator/BenchmarkHashBuildAndJoinOperators.java @@ -225,6 +225,7 @@ public void setup() lookupSourceFactory, types, false, + false, hashChannels, hashChannel, Optional.of(outputChannels), diff --git a/core/trino-main/src/test/java/io/trino/operator/TestHashJoinOperator.java b/core/trino-main/src/test/java/io/trino/operator/TestHashJoinOperator.java index ea05bc47f2df..9ec7c5bf8b60 100644 --- a/core/trino-main/src/test/java/io/trino/operator/TestHashJoinOperator.java +++ b/core/trino-main/src/test/java/io/trino/operator/TestHashJoinOperator.java @@ -243,6 +243,7 @@ public void testUnwrapsLazyBlocks() lookupSourceFactory, probePages.getTypes(), false, + false, Ints.asList(0), getHashChannelAsInt(probePages), Optional.empty(), @@ -295,6 +296,7 @@ public void testYield() lookupSourceFactory, probePages.getTypes(), false, + false, Ints.asList(0), getHashChannelAsInt(probePages), Optional.empty(), @@ -393,7 +395,6 @@ public void testInnerJoinWithSpill(boolean probeHashEnabled, List whe @Test(dataProvider = "joinWithFailingSpillValues") public void testInnerJoinWithFailingSpill(boolean probeHashEnabled, List whenSpill, WhenSpillFails whenSpillFails, boolean isDictionaryProcessingJoinEnabled) - throws Exception { DummySpillerFactory buildSpillerFactory = new DummySpillerFactory(); DummySpillerFactory joinSpillerFactory = new DummySpillerFactory(); @@ -601,6 +602,41 @@ private static MaterializedResult getProperColumns(Operator joinOperator, List lookupSourceFactoryManager = buildSideSetup.getLookupSourceFactoryManager(); + PartitionedLookupSourceFactory lookupSourceFactory = lookupSourceFactoryManager.getJoinBridge(Lifespan.taskWide()); + + // finish probe before any build partition is spilled + lookupSourceFactory.finishProbeOperator(OptionalInt.of(1)); + + // spill build partition after probe is finished + HashBuilderOperator hashBuilderOperator = buildSideSetup.getBuildOperators().get(0); + hashBuilderOperator.startMemoryRevoke().get(); + hashBuilderOperator.finishMemoryRevoke(); + hashBuilderOperator.finish(); + + // hash builder operator should not deadlock waiting for spilled lookup source to be disposed + hashBuilderOperator.isBlocked().get(); + + lookupSourceFactory.destroy(); + assertTrue(hashBuilderOperator.isFinished()); + } + @Test(dataProvider = "hashJoinTestValues") public void testInnerJoinWithNullProbe(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) { @@ -1146,6 +1182,7 @@ public void testInnerJoinWithEmptyLookupSource(boolean parallelBuild, boolean pr lookupSourceFactoryManager, probePages.getTypes(), false, + false, Ints.asList(0), getHashChannelAsInt(probePages), Optional.empty(), @@ -1183,6 +1220,7 @@ public void testLookupOuterJoinWithEmptyLookupSource(boolean parallelBuild, bool new PlanNodeId("test"), lookupSourceFactoryManager, probePages.getTypes(), + false, Ints.asList(0), getHashChannelAsInt(probePages), Optional.empty(), @@ -1319,6 +1357,7 @@ public void testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(boolean parallelB lookupSourceFactoryManager, probePages.getTypes(), false, + false, Ints.asList(0), getHashChannelAsInt(probePages), Optional.empty(), @@ -1339,16 +1378,32 @@ public void testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(boolean parallelB public void testInnerJoinWithBlockingLookupSourceAndEmptyProbe(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) throws Exception { + // join that waits for build side to be collected TaskContext taskContext = createTaskContext(); - OperatorFactory joinOperatorFactory = createJoinOperatorFactoryWithBlockingLookupSource(taskContext, parallelBuild, probeHashEnabled, buildHashEnabled); - + OperatorFactory joinOperatorFactory = createJoinOperatorFactoryWithBlockingLookupSource(taskContext, parallelBuild, probeHashEnabled, buildHashEnabled, true); DriverContext driverContext = taskContext.addPipelineContext(0, true, true, false).addDriverContext(); try (Operator joinOperator = joinOperatorFactory.createOperator(driverContext)) { joinOperatorFactory.noMoreOperators(); assertFalse(joinOperator.needsInput()); joinOperator.finish(); - // lookup join operator will yield once before finishing assertNull(joinOperator.getOutput()); + + // lookup join operator got blocked waiting for build side + assertFalse(joinOperator.isBlocked().isDone()); + assertFalse(joinOperator.isFinished()); + } + + // join that doesn't wait for build side to be collected + taskContext = createTaskContext(); + joinOperatorFactory = createJoinOperatorFactoryWithBlockingLookupSource(taskContext, parallelBuild, probeHashEnabled, buildHashEnabled, false); + driverContext = taskContext.addPipelineContext(0, true, true, false).addDriverContext(); + try (Operator joinOperator = joinOperatorFactory.createOperator(driverContext)) { + joinOperatorFactory.noMoreOperators(); + assertTrue(joinOperator.needsInput()); + joinOperator.finish(); + assertNull(joinOperator.getOutput()); + + // lookup join operator will yield once before finishing assertNull(joinOperator.getOutput()); assertTrue(joinOperator.isBlocked().isDone()); assertTrue(joinOperator.isFinished()); @@ -1359,14 +1414,38 @@ public void testInnerJoinWithBlockingLookupSourceAndEmptyProbe(boolean parallelB public void testInnerJoinWithBlockingLookupSource(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) throws Exception { - TaskContext taskContext = createTaskContext(); - OperatorFactory joinOperatorFactory = createJoinOperatorFactoryWithBlockingLookupSource(taskContext, parallelBuild, probeHashEnabled, buildHashEnabled); + RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), ImmutableList.of(VARCHAR)); + Page probePage = getOnlyElement(probePages.addSequencePage(1, 0).build()); + // join that waits for build side to be collected + TaskContext taskContext = createTaskContext(); + OperatorFactory joinOperatorFactory = createJoinOperatorFactoryWithBlockingLookupSource(taskContext, parallelBuild, probeHashEnabled, buildHashEnabled, true); DriverContext driverContext = taskContext.addPipelineContext(0, true, true, false).addDriverContext(); try (Operator joinOperator = joinOperatorFactory.createOperator(driverContext)) { joinOperatorFactory.noMoreOperators(); assertFalse(joinOperator.needsInput()); assertNull(joinOperator.getOutput()); + + // lookup join operator got blocked waiting for build side + assertFalse(joinOperator.isBlocked().isDone()); + assertFalse(joinOperator.isFinished()); + } + + // join that doesn't wait for build side to be collected + taskContext = createTaskContext(); + joinOperatorFactory = createJoinOperatorFactoryWithBlockingLookupSource(taskContext, parallelBuild, probeHashEnabled, buildHashEnabled, false); + driverContext = taskContext.addPipelineContext(0, true, true, false).addDriverContext(); + try (Operator joinOperator = joinOperatorFactory.createOperator(driverContext)) { + joinOperatorFactory.noMoreOperators(); + assertTrue(joinOperator.needsInput()); + assertNull(joinOperator.getOutput()); + + // join needs input page + assertTrue(joinOperator.isBlocked().isDone()); + assertFalse(joinOperator.isFinished()); + joinOperator.addInput(probePage); + assertNull(joinOperator.getOutput()); + // lookup join operator got blocked waiting for build side assertFalse(joinOperator.isBlocked().isDone()); assertFalse(joinOperator.isFinished()); @@ -1453,7 +1532,7 @@ public void testInnerJoinLoadsPagesInOrder() assertTrue(outputPages.isFinished()); } - private OperatorFactory createJoinOperatorFactoryWithBlockingLookupSource(TaskContext taskContext, boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + private OperatorFactory createJoinOperatorFactoryWithBlockingLookupSource(TaskContext taskContext, boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean waitForBuild) { // build factory List buildTypes = ImmutableList.of(VARCHAR); @@ -1470,6 +1549,7 @@ private OperatorFactory createJoinOperatorFactoryWithBlockingLookupSource(TaskCo lookupSourceFactoryManager, probePages.getTypes(), false, + waitForBuild, Ints.asList(0), getHashChannelAsInt(probePages), Optional.empty(), @@ -1543,6 +1623,7 @@ private OperatorFactory innerJoinOperatorFactory( lookupSourceFactoryManager, probePages.getTypes(), outputSingleMatch, + false, Ints.asList(0), getHashChannelAsInt(probePages), Optional.empty(), diff --git a/core/trino-main/src/test/java/io/trino/operator/TestOperatorAssertion.java b/core/trino-main/src/test/java/io/trino/operator/TestOperatorAssertion.java index 1eb3e31b39f8..4df5cc858163 100644 --- a/core/trino-main/src/test/java/io/trino/operator/TestOperatorAssertion.java +++ b/core/trino-main/src/test/java/io/trino/operator/TestOperatorAssertion.java @@ -25,11 +25,11 @@ import java.util.List; import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; import static java.util.Collections.emptyIterator; import static java.util.Objects.requireNonNull; import static java.util.concurrent.Executors.newSingleThreadScheduledExecutor; +import static java.util.concurrent.TimeUnit.MILLISECONDS; public class TestOperatorAssertion { @@ -50,7 +50,7 @@ public void tearDown() @Test public void testToPagesWithBlockedOperator() { - Operator operator = new BlockedOperator(Duration.valueOf("15 ms")); + Operator operator = new BlockedOperator(new Duration(15, MILLISECONDS)); List pages = OperatorAssertion.toPages(operator, emptyIterator()); Assert.assertEquals(pages, ImmutableList.of()); } @@ -99,7 +99,7 @@ public void finish() if (this.isBlocked == NOT_BLOCKED) { SettableFuture isBlocked = SettableFuture.create(); this.isBlocked = isBlocked; - executor.schedule(() -> isBlocked.set(null), unblockAfter.toMillis(), TimeUnit.MILLISECONDS); + executor.schedule(() -> isBlocked.set(null), unblockAfter.toMillis(), MILLISECONDS); } } diff --git a/core/trino-main/src/test/java/io/trino/operator/scalar/TestDateTimeFunctions.java b/core/trino-main/src/test/java/io/trino/operator/scalar/TestDateTimeFunctions.java index 72498cb6f096..d7e754875098 100644 --- a/core/trino-main/src/test/java/io/trino/operator/scalar/TestDateTimeFunctions.java +++ b/core/trino-main/src/test/java/io/trino/operator/scalar/TestDateTimeFunctions.java @@ -18,7 +18,6 @@ import io.trino.spi.type.BigintType; import io.trino.spi.type.DateType; import io.trino.spi.type.SqlDate; -import io.trino.spi.type.SqlTimestamp; import io.trino.spi.type.SqlTimestampWithTimeZone; import io.trino.spi.type.TimeType; import io.trino.spi.type.TimeZoneKey; @@ -49,15 +48,14 @@ import static io.trino.operator.scalar.DateTimeFunctions.currentDate; import static io.trino.spi.StandardErrorCode.INVALID_FUNCTION_ARGUMENT; import static io.trino.spi.type.BigintType.BIGINT; -import static io.trino.spi.type.DoubleType.DOUBLE; import static io.trino.spi.type.TimeType.createTimeType; import static io.trino.spi.type.TimeWithTimeZoneType.TIME_WITH_TIME_ZONE; import static io.trino.spi.type.TimeZoneKey.UTC_KEY; import static io.trino.spi.type.TimeZoneKey.getTimeZoneKey; import static io.trino.spi.type.TimeZoneKey.getTimeZoneKeyForOffset; import static io.trino.spi.type.TimestampType.TIMESTAMP_MILLIS; -import static io.trino.spi.type.TimestampType.TIMESTAMP_NANOS; import static io.trino.spi.type.TimestampType.createTimestampType; +import static io.trino.spi.type.TimestampWithTimeZoneType.TIMESTAMP_TZ_MILLIS; import static io.trino.spi.type.TimestampWithTimeZoneType.TIMESTAMP_TZ_NANOS; import static io.trino.spi.type.TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE; import static io.trino.spi.type.TimestampWithTimeZoneType.createTimestampWithTimeZoneType; @@ -216,41 +214,42 @@ public void testFromUnixTime() { DateTime dateTime = new DateTime(2001, 1, 22, 3, 4, 5, 0, DATE_TIME_ZONE); double seconds = dateTime.getMillis() / 1000.0; - assertFunction("from_unixtime(" + seconds + ")", TIMESTAMP_MILLIS, sqlTimestampOf(dateTime)); + assertFunction("from_unixtime(" + seconds + ")", TIMESTAMP_TZ_MILLIS, SqlTimestampWithTimeZone.newInstance(3, dateTime.getMillis(), 0, TIME_ZONE_KEY)); dateTime = new DateTime(2001, 1, 22, 3, 4, 5, 888, DATE_TIME_ZONE); seconds = dateTime.getMillis() / 1000.0; - assertFunction("from_unixtime(" + seconds + ")", TIMESTAMP_MILLIS, sqlTimestampOf(dateTime)); + assertFunction("from_unixtime(" + seconds + ")", TIMESTAMP_TZ_MILLIS, SqlTimestampWithTimeZone.newInstance(3, dateTime.getMillis(), 0, TIME_ZONE_KEY)); } @Test public void testFromUnixTimeNanos() { // long - assertFunction("from_unixtime_nanos(1234567890123456789)", TIMESTAMP_NANOS, SqlTimestamp.newInstance(9, 1234567890_123456L, 789000)); - assertFunction("from_unixtime_nanos(999999999)", TIMESTAMP_NANOS, SqlTimestamp.newInstance(9, 999999, 999000)); - assertFunction("from_unixtime_nanos(-1234567890123456789)", TIMESTAMP_NANOS, SqlTimestamp.newInstance(9, -1234567890_123457L, 211000)); - assertFunction("from_unixtime_nanos(-999999999)", TIMESTAMP_NANOS, SqlTimestamp.newInstance(9, -1000000, 1000)); + assertFunction("from_unixtime_nanos(1234567890123456789)", TIMESTAMP_TZ_NANOS, SqlTimestampWithTimeZone.newInstance(9, 1234567890_123L, 456789000, TIME_ZONE_KEY)); + assertFunction("from_unixtime_nanos(999999999)", TIMESTAMP_TZ_NANOS, SqlTimestampWithTimeZone.newInstance(9, 999, 999999000, TIME_ZONE_KEY)); + assertFunction("from_unixtime_nanos(-1234567890123456789)", TIMESTAMP_TZ_NANOS, SqlTimestampWithTimeZone.newInstance(9, -1234567890_124L, 543211000, TIME_ZONE_KEY)); + assertFunction("from_unixtime_nanos(-999999999)", TIMESTAMP_TZ_NANOS, SqlTimestampWithTimeZone.newInstance(9, -1000, 1000, TIME_ZONE_KEY)); // short decimal - assertFunction("from_unixtime_nanos(DECIMAL '1234')", TIMESTAMP_NANOS, SqlTimestamp.newInstance(9, 1, 234000)); - assertFunction("from_unixtime_nanos(DECIMAL '1234.0')", TIMESTAMP_NANOS, SqlTimestamp.newInstance(9, 1, 234000)); - assertFunction("from_unixtime_nanos(DECIMAL '1234.499')", TIMESTAMP_NANOS, SqlTimestamp.newInstance(9, 1, 234000)); - assertFunction("from_unixtime_nanos(DECIMAL '1234.500')", TIMESTAMP_NANOS, SqlTimestamp.newInstance(9, 1, 235000)); - assertFunction("from_unixtime_nanos(DECIMAL '-1234')", TIMESTAMP_NANOS, SqlTimestamp.newInstance(9, -2, 766000)); - assertFunction("from_unixtime_nanos(DECIMAL '-1234.0')", TIMESTAMP_NANOS, SqlTimestamp.newInstance(9, -2, 766000)); - assertFunction("from_unixtime_nanos(DECIMAL '-1234.499')", TIMESTAMP_NANOS, SqlTimestamp.newInstance(9, -2, 766000)); - assertFunction("from_unixtime_nanos(DECIMAL '-1234.500')", TIMESTAMP_NANOS, SqlTimestamp.newInstance(9, -2, 765000)); + assertFunction("from_unixtime_nanos(DECIMAL '1234')", TIMESTAMP_TZ_NANOS, SqlTimestampWithTimeZone.newInstance(9, 0, 1234000, TIME_ZONE_KEY)); + assertFunction("from_unixtime_nanos(DECIMAL '1234.0')", TIMESTAMP_TZ_NANOS, SqlTimestampWithTimeZone.newInstance(9, 0, 1234000, TIME_ZONE_KEY)); + assertFunction("from_unixtime_nanos(DECIMAL '1234.499')", TIMESTAMP_TZ_NANOS, SqlTimestampWithTimeZone.newInstance(9, 0, 1234000, TIME_ZONE_KEY)); + assertFunction("from_unixtime_nanos(DECIMAL '1234.500')", TIMESTAMP_TZ_NANOS, SqlTimestampWithTimeZone.newInstance(9, 0, 1235000, TIME_ZONE_KEY)); + + assertFunction("from_unixtime_nanos(DECIMAL '-1234')", TIMESTAMP_TZ_NANOS, SqlTimestampWithTimeZone.newInstance(9, -1, 998766000, TIME_ZONE_KEY)); + assertFunction("from_unixtime_nanos(DECIMAL '-1234.0')", TIMESTAMP_TZ_NANOS, SqlTimestampWithTimeZone.newInstance(9, -1, 998766000, TIME_ZONE_KEY)); + assertFunction("from_unixtime_nanos(DECIMAL '-1234.499')", TIMESTAMP_TZ_NANOS, SqlTimestampWithTimeZone.newInstance(9, -1, 998766000, TIME_ZONE_KEY)); + assertFunction("from_unixtime_nanos(DECIMAL '-1234.500')", TIMESTAMP_TZ_NANOS, SqlTimestampWithTimeZone.newInstance(9, -1, 998765000, TIME_ZONE_KEY)); // long decimal - assertFunction("from_unixtime_nanos(DECIMAL '12345678900123456789')", TIMESTAMP_NANOS, SqlTimestamp.newInstance(9, 12345678900_123456L, 789000)); - assertFunction("from_unixtime_nanos(DECIMAL '12345678900123456789.000000')", TIMESTAMP_NANOS, SqlTimestamp.newInstance(9, 12345678900_123456L, 789000)); - assertFunction("from_unixtime_nanos(DECIMAL '12345678900123456789.499')", TIMESTAMP_NANOS, SqlTimestamp.newInstance(9, 12345678900_123456L, 789000)); - assertFunction("from_unixtime_nanos(DECIMAL '12345678900123456789.500')", TIMESTAMP_NANOS, SqlTimestamp.newInstance(9, 12345678900_123456L, 790000)); - assertFunction("from_unixtime_nanos(DECIMAL '-12345678900123456789')", TIMESTAMP_NANOS, SqlTimestamp.newInstance(9, -12345678900_123457L, 211000)); - assertFunction("from_unixtime_nanos(DECIMAL '-12345678900123456789.000000')", TIMESTAMP_NANOS, SqlTimestamp.newInstance(9, -12345678900_123457L, 211000)); - assertFunction("from_unixtime_nanos(DECIMAL '-12345678900123456789.499')", TIMESTAMP_NANOS, SqlTimestamp.newInstance(9, -12345678900_123457L, 211000)); - assertFunction("from_unixtime_nanos(DECIMAL '-12345678900123456789.500')", TIMESTAMP_NANOS, SqlTimestamp.newInstance(9, -12345678900_123457L, 210000)); + assertFunction("from_unixtime_nanos(DECIMAL '12345678900123456789')", TIMESTAMP_TZ_NANOS, SqlTimestampWithTimeZone.newInstance(9, 12345678900_123L, 456789000, TIME_ZONE_KEY)); + assertFunction("from_unixtime_nanos(DECIMAL '12345678900123456789.000000')", TIMESTAMP_TZ_NANOS, SqlTimestampWithTimeZone.newInstance(9, 12345678900_123L, 456789000, TIME_ZONE_KEY)); + assertFunction("from_unixtime_nanos(DECIMAL '12345678900123456789.499')", TIMESTAMP_TZ_NANOS, SqlTimestampWithTimeZone.newInstance(9, 12345678900_123L, 456789000, TIME_ZONE_KEY)); + assertFunction("from_unixtime_nanos(DECIMAL '12345678900123456789.500')", TIMESTAMP_TZ_NANOS, SqlTimestampWithTimeZone.newInstance(9, 12345678900_123L, 456790000, TIME_ZONE_KEY)); + assertFunction("from_unixtime_nanos(DECIMAL '-12345678900123456789')", TIMESTAMP_TZ_NANOS, SqlTimestampWithTimeZone.newInstance(9, -12345678900_124L, 543211000, TIME_ZONE_KEY)); + assertFunction("from_unixtime_nanos(DECIMAL '-12345678900123456789.000000')", TIMESTAMP_TZ_NANOS, SqlTimestampWithTimeZone.newInstance(9, -12345678900_124L, 543211000, TIME_ZONE_KEY)); + assertFunction("from_unixtime_nanos(DECIMAL '-12345678900123456789.499')", TIMESTAMP_TZ_NANOS, SqlTimestampWithTimeZone.newInstance(9, -12345678900_124L, 543211000, TIME_ZONE_KEY)); + assertFunction("from_unixtime_nanos(DECIMAL '-12345678900123456789.500')", TIMESTAMP_TZ_NANOS, SqlTimestampWithTimeZone.newInstance(9, -12345678900_124L, 543210000, TIME_ZONE_KEY)); } @Test @@ -299,13 +298,6 @@ public void testFromUnixTimeWithTimeZone() assertFunction(format("from_unixtime(7200, '%s')", zoneId), TIMESTAMP_WITH_TIME_ZONE, toTimestampWithTimeZone(expected)); } - @Test - public void testToUnixTime() - { - assertFunction("to_unixtime(" + TIMESTAMP_LITERAL + ")", DOUBLE, TIMESTAMP.getMillis() / 1000.0); - assertFunction("to_unixtime(" + WEIRD_TIMESTAMP_LITERAL + ")", DOUBLE, WEIRD_TIMESTAMP.getMillis() / 1000.0); - } - @Test public void testDate() { diff --git a/core/trino-main/src/test/java/io/trino/operator/scalar/timestamp/TestTimestamp.java b/core/trino-main/src/test/java/io/trino/operator/scalar/timestamp/TestTimestamp.java index 17117340c963..0519e9845028 100644 --- a/core/trino-main/src/test/java/io/trino/operator/scalar/timestamp/TestTimestamp.java +++ b/core/trino-main/src/test/java/io/trino/operator/scalar/timestamp/TestTimestamp.java @@ -63,19 +63,20 @@ public void teardown() @Test public void testToUnixTime() { - assertThat(assertions.expression("to_unixtime(TIMESTAMP '2020-05-10 12:34:56')")).matches("1589114096e0"); - assertThat(assertions.expression("to_unixtime(TIMESTAMP '2020-05-10 12:34:56.1')")).matches("1589114096.1e0"); - assertThat(assertions.expression("to_unixtime(TIMESTAMP '2020-05-10 12:34:56.12')")).matches("1589114096.12e0"); - assertThat(assertions.expression("to_unixtime(TIMESTAMP '2020-05-10 12:34:56.123')")).matches("1589114096.123e0"); - assertThat(assertions.expression("to_unixtime(TIMESTAMP '2020-05-10 12:34:56.1234')")).matches("1589114096.1234e0"); - assertThat(assertions.expression("to_unixtime(TIMESTAMP '2020-05-10 12:34:56.12345')")).matches("1589114096.12345e0"); - assertThat(assertions.expression("to_unixtime(TIMESTAMP '2020-05-10 12:34:56.123456')")).matches("1589114096.123456e0"); - assertThat(assertions.expression("to_unixtime(TIMESTAMP '2020-05-10 12:34:56.1234567')")).matches("1589114096.1234567e0"); - assertThat(assertions.expression("to_unixtime(TIMESTAMP '2020-05-10 12:34:56.12345678')")).matches("1589114096.1234567e0"); - assertThat(assertions.expression("to_unixtime(TIMESTAMP '2020-05-10 12:34:56.123456789')")).matches("1589114096.1234567e0"); - assertThat(assertions.expression("to_unixtime(TIMESTAMP '2020-05-10 12:34:56.1234567890')")).matches("1589114096.1234567e0"); - assertThat(assertions.expression("to_unixtime(TIMESTAMP '2020-05-10 12:34:56.12345678901')")).matches("1589114096.1234567e0"); - assertThat(assertions.expression("to_unixtime(TIMESTAMP '2020-05-10 12:34:56.123456789012')")).matches("1589114096.1234567e0"); + // to_unixtime is defined for timestamp(p) with time zone, so here we test to_unixtime + required implicit casts + assertThat(assertions.expression("to_unixtime(TIMESTAMP '2020-05-10 12:34:56')")).matches("1589067296e0"); + assertThat(assertions.expression("to_unixtime(TIMESTAMP '2020-05-10 12:34:56.1')")).matches("1589067296.1e0"); + assertThat(assertions.expression("to_unixtime(TIMESTAMP '2020-05-10 12:34:56.12')")).matches("1589067296.12e0"); + assertThat(assertions.expression("to_unixtime(TIMESTAMP '2020-05-10 12:34:56.123')")).matches("1589067296.123e0"); + assertThat(assertions.expression("to_unixtime(TIMESTAMP '2020-05-10 12:34:56.1234')")).matches("1589067296.1234e0"); + assertThat(assertions.expression("to_unixtime(TIMESTAMP '2020-05-10 12:34:56.12345')")).matches("1589067296.1234498e0"); + assertThat(assertions.expression("to_unixtime(TIMESTAMP '2020-05-10 12:34:56.123456')")).matches("1589067296.123456e0"); + assertThat(assertions.expression("to_unixtime(TIMESTAMP '2020-05-10 12:34:56.1234567')")).matches("1589067296.1234567e0"); + assertThat(assertions.expression("to_unixtime(TIMESTAMP '2020-05-10 12:34:56.12345678')")).matches("1589067296.1234567e0"); + assertThat(assertions.expression("to_unixtime(TIMESTAMP '2020-05-10 12:34:56.123456789')")).matches("1589067296.1234567e0"); + assertThat(assertions.expression("to_unixtime(TIMESTAMP '2020-05-10 12:34:56.1234567890')")).matches("1589067296.1234567e0"); + assertThat(assertions.expression("to_unixtime(TIMESTAMP '2020-05-10 12:34:56.12345678901')")).matches("1589067296.1234567e0"); + assertThat(assertions.expression("to_unixtime(TIMESTAMP '2020-05-10 12:34:56.123456789012')")).matches("1589067296.1234567e0"); } @Test diff --git a/core/trino-main/src/test/java/io/trino/server/TestBasicQueryInfo.java b/core/trino-main/src/test/java/io/trino/server/TestBasicQueryInfo.java index a4145a4eb655..01645507d688 100644 --- a/core/trino-main/src/test/java/io/trino/server/TestBasicQueryInfo.java +++ b/core/trino-main/src/test/java/io/trino/server/TestBasicQueryInfo.java @@ -36,6 +36,8 @@ import static io.trino.SessionTestUtils.TEST_SESSION; import static io.trino.execution.QueryState.RUNNING; import static io.trino.server.DynamicFilterService.DynamicFiltersStats; +import static java.util.concurrent.TimeUnit.MINUTES; +import static java.util.concurrent.TimeUnit.SECONDS; import static org.testng.Assert.assertEquals; public class TestBasicQueryInfo @@ -59,14 +61,14 @@ public void testConstructor() DateTime.parse("1991-09-06T05:01-05:30"), DateTime.parse("1991-09-06T05:02-05:30"), DateTime.parse("1991-09-06T06:00-05:30"), - Duration.valueOf("8m"), - Duration.valueOf("7m"), - Duration.valueOf("34m"), - Duration.valueOf("35m"), - Duration.valueOf("44m"), - Duration.valueOf("9m"), - Duration.valueOf("99s"), - Duration.valueOf("12m"), + new Duration(8, MINUTES), + new Duration(7, MINUTES), + new Duration(35, MINUTES), + new Duration(35, MINUTES), + new Duration(44, MINUTES), + new Duration(9, MINUTES), + new Duration(99, SECONDS), + new Duration(12, MINUTES), 13, 14, 15, @@ -87,14 +89,14 @@ public void testConstructor() DataSize.valueOf("28GB"), DataSize.valueOf("29GB"), true, - Duration.valueOf("23m"), - Duration.valueOf("24m"), - Duration.valueOf("26m"), + new Duration(23, MINUTES), + new Duration(24, MINUTES), + new Duration(26, MINUTES), true, ImmutableSet.of(BlockedReason.WAITING_FOR_MEMORY), DataSize.valueOf("271GB"), 281, - Duration.valueOf("20m"), + new Duration(20, MINUTES), DataSize.valueOf("272GB"), 282, DataSize.valueOf("27GB"), @@ -146,8 +148,8 @@ public void testConstructor() assertEquals(basicInfo.getQueryStats().getCreateTime(), DateTime.parse("1991-09-06T05:00-05:30")); assertEquals(basicInfo.getQueryStats().getEndTime(), DateTime.parse("1991-09-06T06:00-05:30")); - assertEquals(basicInfo.getQueryStats().getElapsedTime(), Duration.valueOf("8m")); - assertEquals(basicInfo.getQueryStats().getExecutionTime(), Duration.valueOf("44m")); + assertEquals(basicInfo.getQueryStats().getElapsedTime(), new Duration(8, MINUTES)); + assertEquals(basicInfo.getQueryStats().getExecutionTime(), new Duration(44, MINUTES)); assertEquals(basicInfo.getQueryStats().getTotalDrivers(), 16); assertEquals(basicInfo.getQueryStats().getQueuedDrivers(), 17); @@ -158,7 +160,7 @@ public void testConstructor() assertEquals(basicInfo.getQueryStats().getUserMemoryReservation(), DataSize.valueOf("21GB")); assertEquals(basicInfo.getQueryStats().getTotalMemoryReservation(), DataSize.valueOf("23GB")); assertEquals(basicInfo.getQueryStats().getPeakUserMemoryReservation(), DataSize.valueOf("24GB")); - assertEquals(basicInfo.getQueryStats().getTotalCpuTime(), Duration.valueOf("24m")); + assertEquals(basicInfo.getQueryStats().getTotalCpuTime(), new Duration(24, MINUTES)); assertEquals(basicInfo.getQueryStats().isFullyBlocked(), true); assertEquals(basicInfo.getQueryStats().getBlockedReasons(), ImmutableSet.of(BlockedReason.WAITING_FOR_MEMORY)); diff --git a/core/trino-main/src/test/java/io/trino/server/TestQueryStateInfo.java b/core/trino-main/src/test/java/io/trino/server/TestQueryStateInfo.java index e147f25bb450..0bb467bdc2e1 100644 --- a/core/trino-main/src/test/java/io/trino/server/TestQueryStateInfo.java +++ b/core/trino-main/src/test/java/io/trino/server/TestQueryStateInfo.java @@ -40,6 +40,8 @@ import static io.trino.server.DynamicFilterService.DynamicFiltersStats; import static io.trino.server.QueryStateInfo.createQueuedQueryStateInfo; import static io.trino.spi.resourcegroups.SchedulingPolicy.WEIGHTED; +import static java.util.concurrent.TimeUnit.MINUTES; +import static java.util.concurrent.TimeUnit.SECONDS; import static org.testng.Assert.assertEquals; public class TestQueryStateInfo @@ -110,14 +112,14 @@ private QueryInfo createQueryInfo(String queryId, QueryState state, String query DateTime.parse("1991-09-06T05:01-05:30"), DateTime.parse("1991-09-06T05:02-05:30"), DateTime.parse("1991-09-06T06:00-05:30"), - Duration.valueOf("10s"), - Duration.valueOf("8m"), - Duration.valueOf("7m"), - Duration.valueOf("34m"), - Duration.valueOf("9m"), - Duration.valueOf("10m"), - Duration.valueOf("11m"), - Duration.valueOf("12m"), + new Duration(10, SECONDS), + new Duration(8, MINUTES), + new Duration(7, MINUTES), + new Duration(34, MINUTES), + new Duration(9, MINUTES), + new Duration(10, MINUTES), + new Duration(11, MINUTES), + new Duration(12, MINUTES), 13, 14, 15, @@ -138,14 +140,14 @@ private QueryInfo createQueryInfo(String queryId, QueryState state, String query DataSize.valueOf("28GB"), DataSize.valueOf("29GB"), true, - Duration.valueOf("23m"), - Duration.valueOf("24m"), - Duration.valueOf("26m"), + new Duration(23, MINUTES), + new Duration(24, MINUTES), + new Duration(26, MINUTES), true, ImmutableSet.of(WAITING_FOR_MEMORY), DataSize.valueOf("271GB"), 281, - Duration.valueOf("26m"), + new Duration(26, MINUTES), DataSize.valueOf("272GB"), 282, DataSize.valueOf("27GB"), diff --git a/core/trino-main/src/test/java/io/trino/server/security/TestPasswordAuthenticatorConfig.java b/core/trino-main/src/test/java/io/trino/server/security/TestPasswordAuthenticatorConfig.java index c13000ee1c35..77bfd715212f 100644 --- a/core/trino-main/src/test/java/io/trino/server/security/TestPasswordAuthenticatorConfig.java +++ b/core/trino-main/src/test/java/io/trino/server/security/TestPasswordAuthenticatorConfig.java @@ -32,7 +32,8 @@ public void testDefaults() { assertRecordedDefaults(recordDefaults(PasswordAuthenticatorConfig.class) .setUserMappingPattern(null) - .setUserMappingFile(null)); + .setUserMappingFile(null) + .setPasswordAuthenticatorFiles("etc/password-authenticator.properties")); } @Test @@ -40,15 +41,19 @@ public void testExplicitPropertyMappings() throws IOException { Path userMappingFile = Files.createTempFile(null, null); + Path config1 = Files.createTempFile(null, null); + Path config2 = Files.createTempFile(null, null); Map properties = new ImmutableMap.Builder() .put("http-server.authentication.password.user-mapping.pattern", "(.*)@something") .put("http-server.authentication.password.user-mapping.file", userMappingFile.toString()) + .put("password-authenticator.config-files", config1.toString() + "," + config2.toString()) .build(); PasswordAuthenticatorConfig expected = new PasswordAuthenticatorConfig() .setUserMappingPattern("(.*)@something") - .setUserMappingFile(userMappingFile.toFile()); + .setUserMappingFile(userMappingFile.toFile()) + .setPasswordAuthenticatorFiles(config1 + "," + config2); assertFullMapping(properties, expected); } diff --git a/core/trino-main/src/test/java/io/trino/server/security/TestPasswordAuthenticatorManager.java b/core/trino-main/src/test/java/io/trino/server/security/TestPasswordAuthenticatorManager.java new file mode 100644 index 000000000000..a88261a806b8 --- /dev/null +++ b/core/trino-main/src/test/java/io/trino/server/security/TestPasswordAuthenticatorManager.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 io.trino.server.security; + +import com.google.common.collect.ImmutableList; +import io.trino.spi.security.AccessDeniedException; +import io.trino.spi.security.BasicPrincipal; +import io.trino.spi.security.PasswordAuthenticator; +import io.trino.spi.security.PasswordAuthenticatorFactory; +import org.testng.annotations.Test; + +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.List; +import java.util.Map; + +import static java.nio.file.Files.createTempFile; +import static java.util.Objects.requireNonNull; +import static org.assertj.core.api.Assertions.assertThat; + +public class TestPasswordAuthenticatorManager +{ + @Test + public void testMultipleConfigFiles() + throws Exception + { + Path config1 = createTempFile("passwordConfig", "1"); + Path config2 = createTempFile("passwordConfig", "2"); + Files.write(config1, ImmutableList.of("password-authenticator.name=type1")); + Files.write(config2, ImmutableList.of("password-authenticator.name=type2")); + + PasswordAuthenticatorManager manager = new PasswordAuthenticatorManager(new PasswordAuthenticatorConfig() + .setPasswordAuthenticatorFiles(config1.toAbsolutePath() + "," + config2.toAbsolutePath())); + manager.setRequired(); + manager.addPasswordAuthenticatorFactory(new TestingPasswordAuthenticatorFactory("type1", "password1")); + manager.addPasswordAuthenticatorFactory(new TestingPasswordAuthenticatorFactory("type2", "password2")); + + manager.loadPasswordAuthenticator(); + + List authenticators = manager.getAuthenticators(); + assertThat(login(authenticators, "password1")).isTrue(); + assertThat(login(authenticators, "password2")).isTrue(); + assertThat(login(authenticators, "wrong_password")).isFalse(); + } + + private boolean login(List authenticators, String password) + { + return authenticators.stream() + .anyMatch(authenticator -> { + try { + authenticator.createAuthenticatedPrincipal("ignore", password); + return true; + } + catch (AccessDeniedException e) { + return false; + } + }); + } + + private static class TestingPasswordAuthenticatorFactory + implements PasswordAuthenticatorFactory + { + private final String name; + private final String password; + + public TestingPasswordAuthenticatorFactory(String name, String password) + { + this.name = requireNonNull(name, "name is null"); + this.password = requireNonNull(password, "password is null"); + } + + @Override + public String getName() + { + return name; + } + + @Override + public PasswordAuthenticator create(Map config) + { + return (user, password) -> { + if (password.equals(this.password)) { + return new BasicPrincipal(user); + } + throw new AccessDeniedException("You shall not pass!"); + }; + } + } +} diff --git a/core/trino-main/src/test/java/io/trino/server/security/TestResourceSecurity.java b/core/trino-main/src/test/java/io/trino/server/security/TestResourceSecurity.java index e80ca3a4899f..ac20caa2dbaf 100644 --- a/core/trino-main/src/test/java/io/trino/server/security/TestResourceSecurity.java +++ b/core/trino-main/src/test/java/io/trino/server/security/TestResourceSecurity.java @@ -53,6 +53,7 @@ import java.net.HttpCookie; import java.net.URI; import java.nio.file.Files; +import java.nio.file.Path; import java.nio.file.Paths; import java.security.Principal; import java.security.PrivateKey; @@ -80,6 +81,7 @@ import static javax.servlet.http.HttpServletResponse.SC_OK; import static javax.servlet.http.HttpServletResponse.SC_UNAUTHORIZED; import static javax.ws.rs.core.HttpHeaders.WWW_AUTHENTICATE; +import static org.assertj.core.api.Assertions.assertThat; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; @@ -98,6 +100,7 @@ public class TestResourceSecurity private static final String TEST_USER = "test-user"; private static final String TEST_USER_LOGIN = TEST_USER + "@allowed"; private static final String TEST_PASSWORD = "test-password"; + private static final String TEST_PASSWORD2 = "test-password-2"; private static final String MANAGEMENT_USER = "management-user"; private static final String MANAGEMENT_USER_LOGIN = MANAGEMENT_USER + "@allowed"; private static final String MANAGEMENT_PASSWORD = "management-password"; @@ -115,9 +118,11 @@ public class TestResourceSecurity } private OkHttpClient client; + private Path passwordConfigDummy; @BeforeClass public void setup() + throws IOException { OkHttpClient.Builder clientBuilder = new OkHttpClient.Builder() .followRedirects(false); @@ -130,6 +135,9 @@ public void setup() Optional.empty(), Optional.empty()); client = clientBuilder.build(); + + passwordConfigDummy = Files.createTempFile("passwordConfigDummy", ""); + passwordConfigDummy.toFile().deleteOnExit(); } @Test @@ -185,11 +193,12 @@ public void testPasswordAuthenticator() try (TestingTrinoServer server = TestingTrinoServer.builder() .setProperties(ImmutableMap.builder() .putAll(SECURE_PROPERTIES) + .put("password-authenticator.config-files", passwordConfigDummy.toString()) .put("http-server.authentication.type", "password") .put("http-server.authentication.password.user-mapping.pattern", ALLOWED_USER_MAPPING_PATTERN) .build()) .build()) { - server.getInstance(Key.get(PasswordAuthenticatorManager.class)).setAuthenticator(TestResourceSecurity::authenticate); + server.getInstance(Key.get(PasswordAuthenticatorManager.class)).setAuthenticators(TestResourceSecurity::authenticate); server.getInstance(Key.get(AccessControlManager.class)).addSystemAccessControl(new TestSystemAccessControl()); HttpServerInfo httpServerInfo = server.getInstance(Key.get(HttpServerInfo.class)); assertAuthenticationDisabled(httpServerInfo.getHttpUri()); @@ -197,6 +206,51 @@ public void testPasswordAuthenticator() } } + @Test + public void testMultiplePasswordAuthenticators() + throws Exception + { + try (TestingTrinoServer server = TestingTrinoServer.builder() + .setProperties(ImmutableMap.builder() + .putAll(SECURE_PROPERTIES) + .put("password-authenticator.config-files", passwordConfigDummy.toString()) + .put("http-server.authentication.type", "password") + .put("http-server.authentication.password.user-mapping.pattern", ALLOWED_USER_MAPPING_PATTERN) + .build()) + .build()) { + server.getInstance(Key.get(PasswordAuthenticatorManager.class)).setAuthenticators(TestResourceSecurity::authenticate, TestResourceSecurity::authenticate2); + server.getInstance(Key.get(AccessControlManager.class)).addSystemAccessControl(new TestSystemAccessControl()); + HttpServerInfo httpServerInfo = server.getInstance(Key.get(HttpServerInfo.class)); + assertAuthenticationDisabled(httpServerInfo.getHttpUri()); + assertPasswordAuthentication(httpServerInfo.getHttpsUri(), TEST_PASSWORD, TEST_PASSWORD2); + } + } + + @Test + public void testMultiplePasswordAuthenticatorsMessages() + throws Exception + { + try (TestingTrinoServer server = TestingTrinoServer.builder() + .setProperties(ImmutableMap.builder() + .putAll(SECURE_PROPERTIES) + .put("password-authenticator.config-files", passwordConfigDummy.toString()) + .put("http-server.authentication.type", "password") + .put("http-server.authentication.password.user-mapping.pattern", ALLOWED_USER_MAPPING_PATTERN) + .build()) + .build()) { + server.getInstance(Key.get(PasswordAuthenticatorManager.class)).setAuthenticators(TestResourceSecurity::authenticate, TestResourceSecurity::authenticate2); + server.getInstance(Key.get(AccessControlManager.class)).addSystemAccessControl(new TestSystemAccessControl()); + HttpServerInfo httpServerInfo = server.getInstance(Key.get(HttpServerInfo.class)); + Request request = new Request.Builder() + .url(getAuthorizedUserLocation(httpServerInfo.getHttpsUri())) + .headers(Headers.of("Authorization", Credentials.basic(TEST_USER_LOGIN, "wrong_password"))) + .build(); + try (Response response = client.newCall(request).execute()) { + assertThat(response.message()).isEqualTo("Access Denied: Invalid credentials | Access Denied: Invalid credentials2"); + } + } + } + @Test public void testPasswordAuthenticatorWithInsecureHttp() throws Exception @@ -204,12 +258,13 @@ public void testPasswordAuthenticatorWithInsecureHttp() try (TestingTrinoServer server = TestingTrinoServer.builder() .setProperties(ImmutableMap.builder() .putAll(SECURE_PROPERTIES) + .put("password-authenticator.config-files", passwordConfigDummy.toString()) .put("http-server.authentication.type", "password") .put("http-server.authentication.allow-insecure-over-http", "true") .put("http-server.authentication.password.user-mapping.pattern", ALLOWED_USER_MAPPING_PATTERN) .build()) .build()) { - server.getInstance(Key.get(PasswordAuthenticatorManager.class)).setAuthenticator(TestResourceSecurity::authenticate); + server.getInstance(Key.get(PasswordAuthenticatorManager.class)).setAuthenticators(TestResourceSecurity::authenticate); server.getInstance(Key.get(AccessControlManager.class)).addSystemAccessControl(new TestSystemAccessControl()); HttpServerInfo httpServerInfo = server.getInstance(Key.get(HttpServerInfo.class)); assertInsecureAuthentication(httpServerInfo.getHttpUri()); @@ -224,13 +279,14 @@ public void testFixedManagerAuthenticatorHttpInsecureEnabledOnly() try (TestingTrinoServer server = TestingTrinoServer.builder() .setProperties(ImmutableMap.builder() .putAll(SECURE_PROPERTIES) + .put("password-authenticator.config-files", passwordConfigDummy.toString()) .put("http-server.authentication.type", "password") .put("http-server.authentication.allow-insecure-over-http", "true") .put("http-server.authentication.password.user-mapping.pattern", ALLOWED_USER_MAPPING_PATTERN) .put("management.user", MANAGEMENT_USER) .build()) .build()) { - server.getInstance(Key.get(PasswordAuthenticatorManager.class)).setAuthenticator(TestResourceSecurity::authenticate); + server.getInstance(Key.get(PasswordAuthenticatorManager.class)).setAuthenticators(TestResourceSecurity::authenticate); server.getInstance(Key.get(AccessControlManager.class)).addSystemAccessControl(new TestSystemAccessControl()); HttpServerInfo httpServerInfo = server.getInstance(Key.get(HttpServerInfo.class)); @@ -246,13 +302,14 @@ public void testFixedManagerAuthenticatorHttpInsecureDisabledOnly() try (TestingTrinoServer server = TestingTrinoServer.builder() .setProperties(ImmutableMap.builder() .putAll(SECURE_PROPERTIES) + .put("password-authenticator.config-files", passwordConfigDummy.toString()) .put("http-server.authentication.type", "password") .put("http-server.authentication.allow-insecure-over-http", "false") .put("http-server.authentication.password.user-mapping.pattern", ALLOWED_USER_MAPPING_PATTERN) .put("management.user", MANAGEMENT_USER) .build()) .build()) { - server.getInstance(Key.get(PasswordAuthenticatorManager.class)).setAuthenticator(TestResourceSecurity::authenticate); + server.getInstance(Key.get(PasswordAuthenticatorManager.class)).setAuthenticators(TestResourceSecurity::authenticate); server.getInstance(Key.get(AccessControlManager.class)).addSystemAccessControl(new TestSystemAccessControl()); HttpServerInfo httpServerInfo = server.getInstance(Key.get(HttpServerInfo.class)); @@ -272,13 +329,14 @@ public void testFixedManagerAuthenticatorHttps() try (TestingTrinoServer server = TestingTrinoServer.builder() .setProperties(ImmutableMap.builder() .putAll(SECURE_PROPERTIES) + .put("password-authenticator.config-files", passwordConfigDummy.toString()) .put("http-server.authentication.type", "password") .put("http-server.authentication.allow-insecure-over-http", "true") .put("management.user", MANAGEMENT_USER) .put("management.user.https-enabled", "true") .build()) .build()) { - server.getInstance(Key.get(PasswordAuthenticatorManager.class)).setAuthenticator(TestResourceSecurity::authenticate); + server.getInstance(Key.get(PasswordAuthenticatorManager.class)).setAuthenticators(TestResourceSecurity::authenticate); server.getInstance(Key.get(AccessControlManager.class)).addSystemAccessControl(new TestSystemAccessControl()); HttpServerInfo httpServerInfo = server.getInstance(Key.get(HttpServerInfo.class)); @@ -594,6 +652,12 @@ private void assertInsecureAuthentication(URI baseUri) private void assertPasswordAuthentication(URI baseUri) throws IOException + { + assertPasswordAuthentication(baseUri, TEST_PASSWORD); + } + + private void assertPasswordAuthentication(URI baseUri, String... allowedPasswords) + throws IOException { // public assertOk(client, getPublicLocation(baseUri)); @@ -601,18 +665,24 @@ private void assertPasswordAuthentication(URI baseUri) assertResponseCode(client, getAuthorizedUserLocation(baseUri), SC_UNAUTHORIZED); assertResponseCode(client, getAuthorizedUserLocation(baseUri), SC_UNAUTHORIZED, TEST_USER_LOGIN, null); assertResponseCode(client, getAuthorizedUserLocation(baseUri), SC_UNAUTHORIZED, TEST_USER_LOGIN, "invalid"); - assertResponseCode(client, getAuthorizedUserLocation(baseUri), SC_OK, TEST_USER_LOGIN, TEST_PASSWORD); + for (String password : allowedPasswords) { + assertResponseCode(client, getAuthorizedUserLocation(baseUri), SC_OK, TEST_USER_LOGIN, password); + } // management assertResponseCode(client, getManagementLocation(baseUri), SC_UNAUTHORIZED); assertResponseCode(client, getManagementLocation(baseUri), SC_UNAUTHORIZED, TEST_USER_LOGIN, null); assertResponseCode(client, getManagementLocation(baseUri), SC_UNAUTHORIZED, TEST_USER_LOGIN, "invalid"); - assertResponseCode(client, getManagementLocation(baseUri), SC_FORBIDDEN, TEST_USER_LOGIN, TEST_PASSWORD); + for (String password : allowedPasswords) { + assertResponseCode(client, getManagementLocation(baseUri), SC_FORBIDDEN, TEST_USER_LOGIN, password); + } assertResponseCode(client, getManagementLocation(baseUri), SC_UNAUTHORIZED, MANAGEMENT_USER_LOGIN, null); assertResponseCode(client, getManagementLocation(baseUri), SC_UNAUTHORIZED, MANAGEMENT_USER_LOGIN, "invalid"); assertResponseCode(client, getManagementLocation(baseUri), SC_OK, MANAGEMENT_USER_LOGIN, MANAGEMENT_PASSWORD); // internal assertResponseCode(client, getInternalLocation(baseUri), SC_FORBIDDEN); - assertResponseCode(client, getInternalLocation(baseUri), SC_FORBIDDEN, TEST_USER_LOGIN, TEST_PASSWORD); + for (String password : allowedPasswords) { + assertResponseCode(client, getInternalLocation(baseUri), SC_FORBIDDEN, TEST_USER_LOGIN, password); + } } private static void assertAuthenticationAutomatic(URI baseUri, OkHttpClient authorizedClient) @@ -736,6 +806,14 @@ private static Principal authenticate(String user, String password) throw new AccessDeniedException("Invalid credentials"); } + private static Principal authenticate2(String user, String password) + { + if ((TEST_USER_LOGIN.equals(user) && TEST_PASSWORD2.equals(password)) || (MANAGEMENT_USER_LOGIN.equals(user) && MANAGEMENT_PASSWORD.equals(password))) { + return new BasicPrincipal(user); + } + throw new AccessDeniedException("Invalid credentials2"); + } + private static class TestSystemAccessControl extends AllowAllSystemAccessControl { diff --git a/core/trino-main/src/test/java/io/trino/server/security/oauth2/TestOAuth2Config.java b/core/trino-main/src/test/java/io/trino/server/security/oauth2/TestOAuth2Config.java index 038b65ba4ffd..17f9bc3642b2 100644 --- a/core/trino-main/src/test/java/io/trino/server/security/oauth2/TestOAuth2Config.java +++ b/core/trino-main/src/test/java/io/trino/server/security/oauth2/TestOAuth2Config.java @@ -25,6 +25,8 @@ import static io.airlift.configuration.testing.ConfigAssertions.assertFullMapping; import static io.airlift.configuration.testing.ConfigAssertions.assertRecordedDefaults; import static io.airlift.configuration.testing.ConfigAssertions.recordDefaults; +import static java.util.concurrent.TimeUnit.MINUTES; +import static java.util.concurrent.TimeUnit.SECONDS; public class TestOAuth2Config { @@ -40,7 +42,7 @@ public void testDefaults() .setClientSecret(null) .setAudience(null) .setScopes("openid") - .setChallengeTimeout(Duration.valueOf("15m")) + .setChallengeTimeout(new Duration(15, MINUTES)) .setPrincipalField("sub") .setUserMappingPattern(null) .setUserMappingFile(null)); @@ -76,7 +78,7 @@ public void testExplicitPropertyMappings() .setAudience("https://127.0.0.1:8443") .setScopes("email, offline") .setPrincipalField("some-field") - .setChallengeTimeout(Duration.valueOf("90s")) + .setChallengeTimeout(new Duration(90, SECONDS)) .setUserMappingPattern("(.*)@something") .setUserMappingFile(userMappingFile.toFile()); diff --git a/core/trino-main/src/test/java/io/trino/server/ui/TestWebUi.java b/core/trino-main/src/test/java/io/trino/server/ui/TestWebUi.java index 54606000dbcd..6b3cc6e72571 100644 --- a/core/trino-main/src/test/java/io/trino/server/ui/TestWebUi.java +++ b/core/trino-main/src/test/java/io/trino/server/ui/TestWebUi.java @@ -51,6 +51,7 @@ import java.net.HttpCookie; import java.net.URI; import java.nio.file.Files; +import java.nio.file.Path; import java.nio.file.Paths; import java.security.Principal; import java.security.PrivateKey; @@ -109,6 +110,7 @@ public class TestWebUi .build(); private static final String TEST_USER = "test-user"; private static final String TEST_PASSWORD = "test-password"; + private static final String TEST_PASSWORD2 = "test-password2"; private static final String HMAC_KEY = Resources.getResource("hmac_key.txt").getPath(); private static final PrivateKey JWK_PRIVATE_KEY; @@ -122,9 +124,11 @@ public class TestWebUi } private OkHttpClient client; + private Path passwordConfigDummy; @BeforeClass public void setup() + throws IOException { OkHttpClient.Builder clientBuilder = new OkHttpClient.Builder() .followRedirects(false); @@ -137,6 +141,9 @@ public void setup() Optional.empty(), Optional.empty()); client = clientBuilder.build(); + + passwordConfigDummy = Files.createTempFile("passwordConfigDummy", ""); + passwordConfigDummy.toFile().deleteOnExit(); } @Test @@ -148,7 +155,7 @@ public void testInsecureAuthenticator() .build()) { HttpServerInfo httpServerInfo = server.getInstance(Key.get(HttpServerInfo.class)); // insecure authenticator takes any username, but does not allow any password - testFormAuthentication(server, httpServerInfo, false); + testFormAuthentication(server, httpServerInfo, false, TEST_PASSWORD); } } @@ -160,32 +167,51 @@ public void testPasswordAuthenticator() .setProperties(ImmutableMap.builder() .putAll(SECURE_PROPERTIES) .put("http-server.authentication.type", "password") + .put("password-authenticator.config-files", passwordConfigDummy.toString()) + .build()) + .build()) { + server.getInstance(Key.get(PasswordAuthenticatorManager.class)).setAuthenticators(TestWebUi::authenticate); + HttpServerInfo httpServerInfo = server.getInstance(Key.get(HttpServerInfo.class)); + testFormAuthentication(server, httpServerInfo, true, TEST_PASSWORD); + } + } + + @Test + public void testMultiplePasswordAuthenticators() + throws Exception + { + try (TestingTrinoServer server = TestingTrinoServer.builder() + .setProperties(ImmutableMap.builder() + .putAll(SECURE_PROPERTIES) + .put("http-server.authentication.type", "password") + .put("password-authenticator.config-files", passwordConfigDummy.toString()) .build()) .build()) { - server.getInstance(Key.get(PasswordAuthenticatorManager.class)).setAuthenticator(TestWebUi::authenticate); + server.getInstance(Key.get(PasswordAuthenticatorManager.class)).setAuthenticators(TestWebUi::authenticate, TestWebUi::authenticate2); HttpServerInfo httpServerInfo = server.getInstance(Key.get(HttpServerInfo.class)); - testFormAuthentication(server, httpServerInfo, true); + testFormAuthentication(server, httpServerInfo, true, TEST_PASSWORD); + testFormAuthentication(server, httpServerInfo, true, TEST_PASSWORD2); } } - private void testFormAuthentication(TestingTrinoServer server, HttpServerInfo httpServerInfo, boolean sendPasswordForHttps) + private void testFormAuthentication(TestingTrinoServer server, HttpServerInfo httpServerInfo, boolean sendPasswordForHttps, String password) throws Exception { testRootRedirect(httpServerInfo.getHttpUri(), client); testRootRedirect(httpServerInfo.getHttpsUri(), client); String nodeId = server.getInstance(Key.get(NodeInfo.class)).getNodeId(); - testWorkerResource(nodeId, httpServerInfo.getHttpUri(), false); - testWorkerResource(nodeId, httpServerInfo.getHttpsUri(), sendPasswordForHttps); + testWorkerResource(nodeId, httpServerInfo.getHttpUri(), password, false); + testWorkerResource(nodeId, httpServerInfo.getHttpsUri(), password, sendPasswordForHttps); testLoggedOut(httpServerInfo.getHttpUri()); testLoggedOut(httpServerInfo.getHttpsUri()); - testLogIn(httpServerInfo.getHttpUri(), false); - testLogIn(httpServerInfo.getHttpsUri(), sendPasswordForHttps); + testLogIn(httpServerInfo.getHttpUri(), password, false); + testLogIn(httpServerInfo.getHttpsUri(), password, sendPasswordForHttps); - testFailedLogin(httpServerInfo.getHttpUri(), false); - testFailedLogin(httpServerInfo.getHttpsUri(), sendPasswordForHttps); + testFailedLogin(httpServerInfo.getHttpUri(), false, password); + testFailedLogin(httpServerInfo.getHttpsUri(), sendPasswordForHttps, password); } private static void testRootRedirect(URI baseUri, OkHttpClient client) @@ -208,7 +234,7 @@ private void testLoggedOut(URI baseUri) assertOk(client, getValidVendorLocation(baseUri)); } - private void testLogIn(URI baseUri, boolean sendPassword) + private void testLogIn(URI baseUri, String password, boolean sendPassword) throws Exception { CookieManager cookieManager = new CookieManager(); @@ -229,7 +255,7 @@ private void testLogIn(URI baseUri, boolean sendPassword) assertThat(body).contains("var hidePassword = true;"); } - logIn(baseUri, client, sendPassword); + logIn(baseUri, client, password, sendPassword); HttpCookie cookie = getOnlyElement(cookieManager.getCookieStore().getCookies()); assertEquals(cookie.getPath(), "/ui"); assertEquals(cookie.getDomain(), baseUri.getHost()); @@ -247,16 +273,16 @@ private void testLogIn(URI baseUri, boolean sendPassword) assertThat(cookieManager.getCookieStore().getCookies()).isEmpty(); } - private void testFailedLogin(URI uri, boolean passwordAllowed) + private void testFailedLogin(URI uri, boolean passwordAllowed, String password) throws IOException { testFailedLogin(uri, Optional.empty(), Optional.empty()); - testFailedLogin(uri, Optional.empty(), Optional.of(TEST_PASSWORD)); + testFailedLogin(uri, Optional.empty(), Optional.of(password)); testFailedLogin(uri, Optional.empty(), Optional.of("unknown")); if (passwordAllowed) { testFailedLogin(uri, Optional.of(TEST_USER), Optional.of("unknown")); - testFailedLogin(uri, Optional.of("unknown"), Optional.of(TEST_PASSWORD)); + testFailedLogin(uri, Optional.of("unknown"), Optional.of(password)); testFailedLogin(uri, Optional.of(TEST_USER), Optional.empty()); testFailedLogin(uri, Optional.of("unknown"), Optional.empty()); } @@ -284,13 +310,13 @@ private void testFailedLogin(URI httpsUrl, Optional username, Optionalbuilder() .putAll(SECURE_PROPERTIES) .put("http-server.authentication.type", "password") + .put("password-authenticator.config-files", passwordConfigDummy.toString()) .build()) .build()) { // a password manager is required, so a secure request will fail @@ -371,7 +396,7 @@ public void testNoPasswordAuthenticator() FormAuthenticator formAuthenticator = server.getInstance(Key.get(FormAuthenticator.class)); assertThatThrownBy(() -> formAuthenticator .isValidCredential(TEST_USER, TEST_USER, true)) - .hasMessage("authenticator was not loaded") + .hasMessage("authenticators were not loaded") .isInstanceOf(IllegalStateException.class); assertTrue(formAuthenticator.isLoginEnabled(true)); } @@ -426,7 +451,7 @@ public void testCertAuthenticator() HttpServerInfo httpServerInfo = server.getInstance(Key.get(HttpServerInfo.class)); String nodeId = server.getInstance(Key.get(NodeInfo.class)).getNodeId(); - testLogIn(httpServerInfo.getHttpUri(), false); + testLogIn(httpServerInfo.getHttpUri(), TEST_PASSWORD, false); testNeverAuthorized(httpServerInfo.getHttpsUri(), client); @@ -458,7 +483,7 @@ public void testJwtAuthenticator() HttpServerInfo httpServerInfo = server.getInstance(Key.get(HttpServerInfo.class)); String nodeId = server.getInstance(Key.get(NodeInfo.class)).getNodeId(); - testLogIn(httpServerInfo.getHttpUri(), false); + testLogIn(httpServerInfo.getHttpUri(), TEST_PASSWORD, false); testNeverAuthorized(httpServerInfo.getHttpsUri(), client); @@ -494,7 +519,7 @@ public void testJwtWithJwkAuthenticator() HttpServerInfo httpServerInfo = server.getInstance(Key.get(HttpServerInfo.class)); String nodeId = server.getInstance(Key.get(NodeInfo.class)).getNodeId(); - testLogIn(httpServerInfo.getHttpUri(), false); + testLogIn(httpServerInfo.getHttpUri(), TEST_PASSWORD, false); testNeverAuthorized(httpServerInfo.getHttpsUri(), client); @@ -836,6 +861,14 @@ private static Principal authenticate(String user, String password) throw new AccessDeniedException("Invalid credentials"); } + private static Principal authenticate2(String user, String password) + { + if (TEST_USER.equals(user) && TEST_PASSWORD2.equals(password)) { + return new BasicPrincipal(user); + } + throw new AccessDeniedException("Invalid credentials"); + } + private static String getUiLocation(URI baseUri) { return getLocation(baseUri, "/ui/"); diff --git a/core/trino-main/src/test/java/io/trino/sql/TestExpressionInterpreter.java b/core/trino-main/src/test/java/io/trino/sql/TestExpressionInterpreter.java index 5f7cd93cf9c4..ddc3b226941e 100644 --- a/core/trino-main/src/test/java/io/trino/sql/TestExpressionInterpreter.java +++ b/core/trino-main/src/test/java/io/trino/sql/TestExpressionInterpreter.java @@ -387,16 +387,16 @@ public void testExtract() DateTime dateTime = new DateTime(2001, 8, 22, 3, 4, 5, 321, UTC); double seconds = dateTime.getMillis() / 1000.0; - assertOptimizedEquals("extract(YEAR FROM from_unixtime(" + seconds + "))", "2001"); - assertOptimizedEquals("extract(QUARTER FROM from_unixtime(" + seconds + "))", "3"); - assertOptimizedEquals("extract(MONTH FROM from_unixtime(" + seconds + "))", "8"); - assertOptimizedEquals("extract(WEEK FROM from_unixtime(" + seconds + "))", "34"); - assertOptimizedEquals("extract(DOW FROM from_unixtime(" + seconds + "))", "3"); - assertOptimizedEquals("extract(DOY FROM from_unixtime(" + seconds + "))", "234"); - assertOptimizedEquals("extract(DAY FROM from_unixtime(" + seconds + "))", "22"); - assertOptimizedEquals("extract(HOUR FROM from_unixtime(" + seconds + "))", "3"); - assertOptimizedEquals("extract(MINUTE FROM from_unixtime(" + seconds + "))", "4"); - assertOptimizedEquals("extract(SECOND FROM from_unixtime(" + seconds + "))", "5"); + assertOptimizedEquals("extract(YEAR FROM from_unixtime(" + seconds + ",'UTC'))", "2001"); + assertOptimizedEquals("extract(QUARTER FROM from_unixtime(" + seconds + ",'UTC'))", "3"); + assertOptimizedEquals("extract(MONTH FROM from_unixtime(" + seconds + ",'UTC'))", "8"); + assertOptimizedEquals("extract(WEEK FROM from_unixtime(" + seconds + ",'UTC'))", "34"); + assertOptimizedEquals("extract(DOW FROM from_unixtime(" + seconds + ",'UTC'))", "3"); + assertOptimizedEquals("extract(DOY FROM from_unixtime(" + seconds + ",'UTC'))", "234"); + assertOptimizedEquals("extract(DAY FROM from_unixtime(" + seconds + ",'UTC'))", "22"); + assertOptimizedEquals("extract(HOUR FROM from_unixtime(" + seconds + ",'UTC'))", "3"); + assertOptimizedEquals("extract(MINUTE FROM from_unixtime(" + seconds + ",'UTC'))", "4"); + assertOptimizedEquals("extract(SECOND FROM from_unixtime(" + seconds + ",'UTC'))", "5"); assertOptimizedEquals("extract(TIMEZONE_HOUR FROM from_unixtime(" + seconds + ", 7, 9))", "7"); assertOptimizedEquals("extract(TIMEZONE_MINUTE FROM from_unixtime(" + seconds + ", 7, 9))", "9"); diff --git a/core/trino-main/src/test/java/io/trino/sql/analyzer/TestAnalyzer.java b/core/trino-main/src/test/java/io/trino/sql/analyzer/TestAnalyzer.java index 2037f1ae8c16..ea968796431f 100644 --- a/core/trino-main/src/test/java/io/trino/sql/analyzer/TestAnalyzer.java +++ b/core/trino-main/src/test/java/io/trino/sql/analyzer/TestAnalyzer.java @@ -15,6 +15,7 @@ import com.google.common.base.Joiner; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import io.trino.Session; import io.trino.SystemSessionProperties; @@ -35,6 +36,7 @@ import io.trino.metadata.Metadata; import io.trino.metadata.QualifiedObjectName; import io.trino.metadata.SessionPropertyManager; +import io.trino.metadata.TableHandle; import io.trino.plugin.base.security.AllowAllSystemAccessControl; import io.trino.security.AccessControl; import io.trino.security.AccessControlConfig; @@ -42,6 +44,8 @@ import io.trino.security.AllowAllAccessControl; import io.trino.spi.connector.ColumnMetadata; import io.trino.spi.connector.Connector; +import io.trino.spi.connector.ConnectorMaterializedViewDefinition; +import io.trino.spi.connector.ConnectorMaterializedViewDefinition.Column; import io.trino.spi.connector.ConnectorMetadata; import io.trino.spi.connector.ConnectorTableMetadata; import io.trino.spi.connector.ConnectorTransactionHandle; @@ -56,6 +60,7 @@ import io.trino.sql.planner.TypeAnalyzer; import io.trino.sql.tree.Statement; import io.trino.testing.TestingMetadata; +import io.trino.testing.TestingMetadata.TestingTableHandle; import io.trino.testing.assertions.TrinoExceptionAssert; import io.trino.transaction.TransactionManager; import org.intellij.lang.annotations.Language; @@ -66,6 +71,7 @@ import java.util.Optional; import java.util.function.Consumer; +import static com.google.common.collect.Iterables.getOnlyElement; import static io.trino.connector.CatalogName.createInformationSchemaCatalogName; import static io.trino.connector.CatalogName.createSystemTablesCatalogName; import static io.trino.cost.StatsCalculatorModule.createNewStatsCalculator; @@ -144,6 +150,7 @@ import static java.util.Collections.emptyList; import static java.util.Collections.emptyMap; import static java.util.Collections.nCopies; +import static org.assertj.core.api.Assertions.assertThat; @Test(singleThreaded = true) public class TestAnalyzer @@ -2465,6 +2472,17 @@ public void testCreateRecursiveView() { assertFails("CREATE OR REPLACE VIEW v1 AS SELECT * FROM v1") .hasErrorCode(VIEW_IS_RECURSIVE); + assertFails("CREATE OR REPLACE VIEW mv1 AS SELECT * FROM mv1") + .hasErrorCode(VIEW_IS_RECURSIVE); + } + + @Test + public void testCreateMaterializedRecursiveView() + { + assertFails("CREATE OR REPLACE MATERIALIZED VIEW v1 AS SELECT * FROM v1") + .hasErrorCode(VIEW_IS_RECURSIVE); + assertFails("CREATE OR REPLACE MATERIALIZED VIEW mv1 AS SELECT * FROM mv1") + .hasErrorCode(VIEW_IS_RECURSIVE); } @Test @@ -2488,6 +2506,44 @@ public void testShowCreateView() .hasErrorCode(TABLE_NOT_FOUND); } + // This test validates object resolution order (materialized view, view and table). + // The order is arbitrary (connector should not return different object types with same name). + // However, "SHOW CREATE" command should be consistent with how object resolution is performed + // during table scan. + @Test + public void testShowCreateDuplicateNames() + { + analyze("SHOW CREATE MATERIALIZED VIEW table_view_and_materialized_view"); + assertFails("SHOW CREATE VIEW table_view_and_materialized_view") + .hasErrorCode(NOT_SUPPORTED) + .hasMessageContaining("Relation 'tpch.s1.table_view_and_materialized_view' is a materialized view, not a view"); + assertFails("SHOW CREATE TABLE table_view_and_materialized_view") + .hasErrorCode(NOT_SUPPORTED) + .hasMessageContaining("Relation 'tpch.s1.table_view_and_materialized_view' is a materialized view, not a table"); + + analyze("SHOW CREATE VIEW table_and_view"); + assertFails("SHOW CREATE TABLE table_and_view") + .hasErrorCode(NOT_SUPPORTED) + .hasMessageContaining("Relation 'tpch.s1.table_and_view' is a view, not a table"); + } + + // This test validates object resolution order (materialized view, view and table). + // The order is arbitrary (connector should not return different object types with same name) + // and can be changed along with test. + @Test + public void testAnalysisDuplicateNames() + { + // Materialized view redirects to "t1" + Analysis analysis = analyze("SELECT * FROM table_view_and_materialized_view"); + TableHandle handle = getOnlyElement(analysis.getTables()); + assertThat(((TestingTableHandle) handle.getConnectorHandle()).getTableName().getTableName()).isEqualTo("t1"); + + // View redirects to "t2" + analysis = analyze("SELECT * FROM table_and_view"); + handle = getOnlyElement(analysis.getTables()); + assertThat(((TestingTableHandle) handle.getConnectorHandle()).getTableName().getTableName()).isEqualTo("t2"); + } + @Test public void testStaleView() { @@ -3034,6 +3090,18 @@ public void setup() new ColumnMetadata("d", new ArrayType(DOUBLE)))), false)); + // materialized view referencing table in same schema + ConnectorMaterializedViewDefinition materializedViewData1 = new ConnectorMaterializedViewDefinition( + "select a from t1", + Optional.empty(), + Optional.of(TPCH_CATALOG), + Optional.of("s1"), + ImmutableList.of(new ConnectorMaterializedViewDefinition.Column("a", BIGINT.getTypeId())), + Optional.of("comment"), + "user", + ImmutableMap.of()); + inSetupTransaction(session -> metadata.createMaterializedView(session, new QualifiedObjectName(TPCH_CATALOG, "s1", "mv1"), materializedViewData1, false, true)); + // valid view referencing table in same schema ConnectorViewDefinition viewData1 = new ConnectorViewDefinition( "select a from t1", @@ -3151,6 +3219,56 @@ public void setup() new ConnectorTableMetadata(t5, ImmutableList.of( new ColumnMetadata("b", singleFieldRowType))), false)); + + QualifiedObjectName tableViewAndMaterializedView = new QualifiedObjectName(TPCH_CATALOG, "s1", "table_view_and_materialized_view"); + inSetupTransaction(session -> metadata.createMaterializedView( + session, + tableViewAndMaterializedView, + new ConnectorMaterializedViewDefinition( + "SELECT a FROM t1", + Optional.of("t1"), + Optional.of(TPCH_CATALOG), + Optional.of("s1"), + ImmutableList.of(new Column("a", BIGINT.getTypeId())), + Optional.empty(), + "some user", + ImmutableMap.of()), + false, + false)); + ConnectorViewDefinition viewDefinition = new ConnectorViewDefinition( + "SELECT a FROM t2", + Optional.of(TPCH_CATALOG), + Optional.of("s1"), + ImmutableList.of(new ViewColumn("a", BIGINT.getTypeId())), + Optional.empty(), + Optional.empty(), + false); + inSetupTransaction(session -> metadata.createView( + session, + tableViewAndMaterializedView, + viewDefinition, + false)); + inSetupTransaction(session -> metadata.createTable( + session, + CATALOG_FOR_IDENTIFIER_CHAIN_TESTS, + new ConnectorTableMetadata( + tableViewAndMaterializedView.asSchemaTableName(), + ImmutableList.of(new ColumnMetadata("a", BIGINT))), + false)); + + QualifiedObjectName tableAndView = new QualifiedObjectName(TPCH_CATALOG, "s1", "table_and_view"); + inSetupTransaction(session -> metadata.createView( + session, + tableAndView, + viewDefinition, + false)); + inSetupTransaction(session -> metadata.createTable( + session, + CATALOG_FOR_IDENTIFIER_CHAIN_TESTS, + new ConnectorTableMetadata( + tableAndView.asSchemaTableName(), + ImmutableList.of(new ColumnMetadata("a", BIGINT))), + false)); } private void inSetupTransaction(Consumer consumer) @@ -3176,21 +3294,21 @@ private static Analyzer createAnalyzer(Session session, Metadata metadata) createNewStatsCalculator(metadata, new TypeAnalyzer(SQL_PARSER, metadata))); } - private void analyze(@Language("SQL") String query) + private Analysis analyze(@Language("SQL") String query) { - analyze(CLIENT_SESSION, query); + return analyze(CLIENT_SESSION, query); } - private void analyze(Session clientSession, @Language("SQL") String query) + private Analysis analyze(Session clientSession, @Language("SQL") String query) { - transaction(transactionManager, accessControl) + return transaction(transactionManager, accessControl) .singleStatement() .readUncommitted() .execute(clientSession, session -> { Analyzer analyzer = createAnalyzer(session, metadata); Statement statement = SQL_PARSER.createStatement(query, new ParsingOptions( new FeaturesConfig().isParseDecimalLiteralsAsDouble() ? AS_DOUBLE : AS_DECIMAL)); - analyzer.analyze(statement); + return analyzer.analyze(statement); }); } diff --git a/core/trino-main/src/test/java/io/trino/sql/analyzer/TestOutput.java b/core/trino-main/src/test/java/io/trino/sql/analyzer/TestOutput.java index 9d1bfc51f7c0..ef110ac23894 100644 --- a/core/trino-main/src/test/java/io/trino/sql/analyzer/TestOutput.java +++ b/core/trino-main/src/test/java/io/trino/sql/analyzer/TestOutput.java @@ -13,9 +13,16 @@ */ package io.trino.sql.analyzer; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; import io.airlift.json.JsonCodec; +import io.trino.execution.Column; +import io.trino.metadata.QualifiedObjectName; +import io.trino.sql.analyzer.Analysis.SourceColumn; import org.testng.annotations.Test; +import java.util.Optional; + import static org.testng.Assert.assertEquals; public class TestOutput @@ -25,7 +32,16 @@ public class TestOutput @Test public void testRoundTrip() { - Output expected = new Output("connectorId", "schema", "table"); + Output expected = new Output( + "connectorId", + "schema", + "table", + Optional.of( + ImmutableList.of( + new OutputColumn( + new Column("column", "type"), + ImmutableSet.of( + new SourceColumn(QualifiedObjectName.valueOf("catalog.schema.table"), "column")))))); String json = codec.toJson(expected); Output actual = codec.fromJson(json); diff --git a/core/trino-main/src/test/java/io/trino/sql/gen/TestExpressionCompiler.java b/core/trino-main/src/test/java/io/trino/sql/gen/TestExpressionCompiler.java index 869ea4dce719..0709d678199a 100644 --- a/core/trino-main/src/test/java/io/trino/sql/gen/TestExpressionCompiler.java +++ b/core/trino-main/src/test/java/io/trino/sql/gen/TestExpressionCompiler.java @@ -1534,7 +1534,7 @@ public void testExtract() expected = callExtractFunction(micros, field); } String expressionPattern = format( - "extract(%s from from_unixtime(cast(%s as double) / 1000000))", + "extract(%s from from_unixtime(cast(%s as double) / 1000000, 'UTC'))", field, micros); assertExecute(generateExpression(expressionPattern, micros), BIGINT, expected); diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/BenchmarkPlanner.java b/core/trino-main/src/test/java/io/trino/sql/planner/BenchmarkPlanner.java index ceb417a4bdbc..9f89ddf58d18 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/BenchmarkPlanner.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/BenchmarkPlanner.java @@ -21,6 +21,7 @@ import io.trino.plugin.tpch.TpchConnectorFactory; import io.trino.testing.LocalQueryRunner; import io.trino.tpch.Customer; +import org.intellij.lang.annotations.Language; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; @@ -53,7 +54,9 @@ import static io.trino.testing.TestingSession.testSessionBuilder; import static java.lang.String.format; import static java.util.Locale.ENGLISH; +import static java.util.stream.Collectors.joining; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; @SuppressWarnings("MethodMayBeStatic") @State(Scope.Benchmark) @@ -73,6 +76,8 @@ public static class BenchmarkData private LocalQueryRunner queryRunner; private List queries; + @Language("SQL") + private String largeInQuery; private Session session; @Setup @@ -93,6 +98,11 @@ public void setup() .filter(i -> i != 15) // q15 has two queries in it .map(i -> readResource(format("/io/trino/tpch/queries/q%d.sql", i))) .collect(toImmutableList()); + + largeInQuery = "SELECT * from orders where o_orderkey in " + + IntStream.range(0, 5000) + .mapToObj(Integer::toString) + .collect(joining(", ", "(", ")")); } @TearDown @@ -125,6 +135,16 @@ public List planQueries(BenchmarkData benchmarkData) }); } + @Benchmark + public Plan planLargeInQuery(BenchmarkData benchmarkData) + { + return benchmarkData.queryRunner.inTransaction(transactionSession -> { + LogicalPlanner.Stage stage = LogicalPlanner.Stage.valueOf(benchmarkData.stage.toUpperCase(ENGLISH)); + return benchmarkData.queryRunner.createPlan( + transactionSession, benchmarkData.largeInQuery, stage, false, WarningCollector.NOOP); + }); + } + @Test public void verify() { @@ -132,6 +152,7 @@ public void verify() data.setup(); BenchmarkPlanner benchmark = new BenchmarkPlanner(); assertEquals(benchmark.planQueries(data).size(), 21); + assertNotNull(benchmark.planLargeInQuery(data)); } public static void main(String[] args) diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPushTopNIntoTableScan.java b/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPushTopNIntoTableScan.java index 9ca4fb06acd7..0e45ff33e0a5 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPushTopNIntoTableScan.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPushTopNIntoTableScan.java @@ -103,7 +103,7 @@ public void testDoesNotFire() } @Test - public void testPushTopNIntoTableScan() + public void testPushSingleTopNIntoTableScan() { try (RuleTester ruleTester = defaultRuleTester()) { MockConnectorTableHandle connectorHandle = new MockConnectorTableHandle(TEST_SCHEMA_TABLE); @@ -135,7 +135,7 @@ public void testPushTopNIntoTableScan() } @Test - public void testPushTopNIntoTableScanPartial() + public void testPushSingleTopNIntoTableScanNotGuaranteed() { try (RuleTester ruleTester = defaultRuleTester()) { MockConnectorTableHandle connectorHandle = new MockConnectorTableHandle(TEST_SCHEMA_TABLE); @@ -170,6 +170,74 @@ dimensionName, equalTo(dimensionColumn), } } + @Test + public void testPushPartialTopNIntoTableScan() + { + try (RuleTester ruleTester = defaultRuleTester()) { + MockConnectorTableHandle connectorHandle = new MockConnectorTableHandle(TEST_SCHEMA_TABLE); + // make the mock connector return a new connectorHandle + MockConnectorFactory.ApplyTopN applyTopN = + (session, handle, topNCount, sortItems, tableAssignments) -> Optional.of(new TopNApplicationResult<>(connectorHandle, true)); + MockConnectorFactory mockFactory = createMockFactory(assignments, Optional.of(applyTopN)); + + ruleTester.getQueryRunner().createCatalog(MOCK_CATALOG, mockFactory, ImmutableMap.of()); + + ruleTester.assertThat(new PushTopNIntoTableScan(ruleTester.getMetadata())) + .on(p -> { + Symbol dimension = p.symbol(dimensionName, VARCHAR); + Symbol metric = p.symbol(metricName, BIGINT); + return p.topN(1, ImmutableList.of(dimension), TopNNode.Step.PARTIAL, + p.tableScan(TEST_TABLE_HANDLE, + ImmutableList.of(dimension, metric), + ImmutableMap.of( + dimension, dimensionColumn, + metric, metricColumn))); + }) + .withSession(MOCK_SESSION) + .matches( + tableScan( + equalTo(connectorHandle), + TupleDomain.all(), + new HashMap<>())); + } + } + + @Test + public void testPushPartialTopNIntoTableScanNotGuaranteed() + { + try (RuleTester ruleTester = defaultRuleTester()) { + MockConnectorTableHandle connectorHandle = new MockConnectorTableHandle(TEST_SCHEMA_TABLE); + // make the mock connector return a new connectorHandle + MockConnectorFactory.ApplyTopN applyTopN = + (session, handle, topNCount, sortItems, tableAssignments) -> Optional.of(new TopNApplicationResult<>(connectorHandle, false)); + MockConnectorFactory mockFactory = createMockFactory(assignments, Optional.of(applyTopN)); + + ruleTester.getQueryRunner().createCatalog(MOCK_CATALOG, mockFactory, ImmutableMap.of()); + + ruleTester.assertThat(new PushTopNIntoTableScan(ruleTester.getMetadata())) + .on(p -> { + Symbol dimension = p.symbol(dimensionName, VARCHAR); + Symbol metric = p.symbol(metricName, BIGINT); + return p.topN(1, ImmutableList.of(dimension), TopNNode.Step.PARTIAL, + p.tableScan(TEST_TABLE_HANDLE, + ImmutableList.of(dimension, metric), + ImmutableMap.of( + dimension, dimensionColumn, + metric, metricColumn))); + }) + .withSession(MOCK_SESSION) + .matches( + topN(1, ImmutableList.of(sort(dimensionName, ASCENDING, FIRST)), + TopNNode.Step.PARTIAL, + tableScan( + equalTo(connectorHandle), + TupleDomain.all(), + ImmutableMap.of( + dimensionName, equalTo(dimensionColumn), + metricName, equalTo(metricColumn))))); + } + } + private MockConnectorFactory createMockFactory(Map assignments, Optional applyTopN) { List metadata = assignments.entrySet().stream() diff --git a/core/trino-parser/pom.xml b/core/trino-parser/pom.xml index fade86cc20d1..14b61b3b61a3 100644 --- a/core/trino-parser/pom.xml +++ b/core/trino-parser/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/core/trino-parser/src/test/java/io/trino/sql/parser/TestSqlParser.java b/core/trino-parser/src/test/java/io/trino/sql/parser/TestSqlParser.java index 940760e172ce..b7a7a18d4d23 100644 --- a/core/trino-parser/src/test/java/io/trino/sql/parser/TestSqlParser.java +++ b/core/trino-parser/src/test/java/io/trino/sql/parser/TestSqlParser.java @@ -2904,18 +2904,30 @@ private static QualifiedName makeQualifiedName(String tableName) return QualifiedName.of(parts); } + /** + * @deprecated use {@link ParserAssert#statement(String)} instead + */ + @Deprecated private static void assertStatement(String query, Statement expected) { assertParsed(query, expected, SQL_PARSER.createStatement(query, new ParsingOptions())); assertFormattedSql(SQL_PARSER, expected); } + /** + * @deprecated use {@link ParserAssert#statement(String)} instead + */ + @Deprecated private static void assertInvalidStatement(String statement, String expectedErrorMessageRegex) { assertThatThrownBy(() -> SQL_PARSER.createStatement(statement, new ParsingOptions())) .isInstanceOfSatisfying(ParsingException.class, e -> assertTrue(e.getErrorMessage().matches(expectedErrorMessageRegex))); } + /** + * @deprecated use {@link ParserAssert#expression(String)} instead + */ + @Deprecated private static void assertExpression(String expression, Expression expected) { requireNonNull(expression, "expression is null"); diff --git a/core/trino-server-main/pom.xml b/core/trino-server-main/pom.xml index 9b99e27f0a78..2e0f58ec6f4d 100644 --- a/core/trino-server-main/pom.xml +++ b/core/trino-server-main/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/core/trino-server-rpm/pom.xml b/core/trino-server-rpm/pom.xml index a223cea45f53..ac39c734f37d 100644 --- a/core/trino-server-rpm/pom.xml +++ b/core/trino-server-rpm/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/core/trino-server/pom.xml b/core/trino-server/pom.xml index 9dc18f4cd3dd..c083fb9e1635 100644 --- a/core/trino-server/pom.xml +++ b/core/trino-server/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/core/trino-spi/pom.xml b/core/trino-spi/pom.xml index bd18c55c1550..5e9e540d09e2 100644 --- a/core/trino-spi/pom.xml +++ b/core/trino-spi/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/core/trino-spi/src/main/java/io/trino/spi/block/RunLengthEncodedBlock.java b/core/trino-spi/src/main/java/io/trino/spi/block/RunLengthEncodedBlock.java index fd27111d5061..7e69b680fdf1 100644 --- a/core/trino-spi/src/main/java/io/trino/spi/block/RunLengthEncodedBlock.java +++ b/core/trino-spi/src/main/java/io/trino/spi/block/RunLengthEncodedBlock.java @@ -282,7 +282,7 @@ public boolean isNull(int position) public String toString() { StringBuilder sb = new StringBuilder(getClass().getSimpleName()); - sb.append("positionCount=").append(positionCount); + sb.append("{positionCount=").append(positionCount); sb.append(", value=").append(value); sb.append('}'); return sb.toString(); diff --git a/core/trino-spi/src/main/java/io/trino/spi/connector/ColumnMetadata.java b/core/trino-spi/src/main/java/io/trino/spi/connector/ColumnMetadata.java index d89af0f487ab..4262a8546614 100644 --- a/core/trino-spi/src/main/java/io/trino/spi/connector/ColumnMetadata.java +++ b/core/trino-spi/src/main/java/io/trino/spi/connector/ColumnMetadata.java @@ -135,6 +135,12 @@ public Map getProperties() return properties; } + public ColumnSchema getColumnSchema() + { + return ColumnSchema.builder(this) + .build(); + } + @Override public String toString() { diff --git a/core/trino-spi/src/main/java/io/trino/spi/connector/ColumnSchema.java b/core/trino-spi/src/main/java/io/trino/spi/connector/ColumnSchema.java new file mode 100644 index 000000000000..e3b19aa8aa1a --- /dev/null +++ b/core/trino-spi/src/main/java/io/trino/spi/connector/ColumnSchema.java @@ -0,0 +1,135 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.trino.spi.connector; + +import io.trino.spi.type.Type; + +import java.util.Objects; + +import static io.trino.spi.connector.SchemaUtil.checkNotEmpty; +import static java.util.Locale.ENGLISH; +import static java.util.Objects.requireNonNull; + +public final class ColumnSchema +{ + private final String name; + private final Type type; + private final boolean hidden; + + private ColumnSchema(String name, Type type, boolean hidden) + { + checkNotEmpty(name, "name"); + requireNonNull(type, "type is null"); + + this.name = name.toLowerCase(ENGLISH); + this.type = type; + this.hidden = hidden; + } + + public String getName() + { + return name; + } + + public Type getType() + { + return type; + } + + public boolean isHidden() + { + return hidden; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ColumnSchema that = (ColumnSchema) o; + return hidden == that.hidden + && name.equals(that.name) + && type.equals(that.type); + } + + @Override + public int hashCode() + { + return Objects.hash(name, type, hidden); + } + + @Override + public String toString() + { + return new StringBuilder("ColumnBasicMetadata{") + .append("name='").append(name).append('\'') + .append(", type=").append(type) + .append(", hidden=").append(hidden) + .append('}') + .toString(); + } + + public static Builder builder() + { + return new Builder(); + } + + public static Builder builder(ColumnMetadata columnMetadata) + { + return new Builder(columnMetadata); + } + + public static class Builder + { + private String name; + private Type type; + private boolean hidden; + + private Builder() {} + + private Builder(ColumnMetadata columnMetadata) + { + this.name = columnMetadata.getName(); + this.type = columnMetadata.getType(); + this.hidden = columnMetadata.isHidden(); + } + + public Builder setName(String name) + { + this.name = requireNonNull(name, "name is null"); + return this; + } + + public Builder setType(Type type) + { + this.type = requireNonNull(type, "type is null"); + return this; + } + + public Builder setHidden(boolean hidden) + { + this.hidden = hidden; + return this; + } + + public ColumnSchema build() + { + return new ColumnSchema(name, type, hidden); + } + } +} diff --git a/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorMaterializedViewDefinition.java b/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorMaterializedViewDefinition.java index ba6da8447999..c8a2719c03c4 100644 --- a/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorMaterializedViewDefinition.java +++ b/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorMaterializedViewDefinition.java @@ -44,15 +44,16 @@ public ConnectorMaterializedViewDefinition( @JsonProperty("schema") Optional schema, @JsonProperty("columns") List columns, @JsonProperty("comment") Optional comment, - @JsonProperty("owner") Optional owner, + @JsonProperty("owner") String owner, @JsonProperty("properties") Map properties) { - this(originalSql, requireNonNull(storageTable, "storageTable is null").orElse(null), catalog, schema, columns, comment, owner, properties); + this(originalSql, requireNonNull(storageTable, "storageTable is null").orElse(null), catalog, schema, columns, comment, Optional.of(owner), properties); } /* * This constructor is for JSON deserialization only. Do not use. */ + // TODO: Simplify this constructor and getters: https://github.com/trinodb/trino/issues/7537 @Deprecated @JsonCreator public ConnectorMaterializedViewDefinition( @@ -80,6 +81,10 @@ public ConnectorMaterializedViewDefinition( if (columns.isEmpty()) { throw new IllegalArgumentException("columns list is empty"); } + + if (owner.isEmpty()) { + throw new IllegalArgumentException("owner must be present"); + } } @JsonProperty diff --git a/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorMetadata.java b/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorMetadata.java index b45c94d916f7..f6bcd3b0c43c 100644 --- a/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorMetadata.java +++ b/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorMetadata.java @@ -161,6 +161,17 @@ default Optional getCommonPartitioningHandle(Connec return Optional.empty(); } + /** + * Return table schema definition for the specified table handle. + * This method is useful when getting full table metadata is expensive. + * + * @throws RuntimeException if table handle is no longer valid + */ + default ConnectorTableSchema getTableSchema(ConnectorSession session, ConnectorTableHandle table) + { + return getTableMetadata(session, table).getTableSchema(); + } + /** * Return the metadata for the specified table handle. * diff --git a/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorTableMetadata.java b/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorTableMetadata.java index d35e8894a53f..cb1963095098 100644 --- a/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorTableMetadata.java +++ b/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorTableMetadata.java @@ -21,6 +21,7 @@ import static java.util.Collections.emptyMap; import static java.util.Objects.requireNonNull; +import static java.util.stream.Collectors.toUnmodifiableList; public class ConnectorTableMetadata { @@ -71,6 +72,15 @@ public Optional getComment() return comment; } + public ConnectorTableSchema getTableSchema() + { + return new ConnectorTableSchema( + table, + columns.stream() + .map(ColumnMetadata::getColumnSchema) + .collect(toUnmodifiableList())); + } + @Override public String toString() { diff --git a/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorTableSchema.java b/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorTableSchema.java new file mode 100644 index 000000000000..afa9694ecb66 --- /dev/null +++ b/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorTableSchema.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 io.trino.spi.connector; + +import java.util.List; + +import static java.util.Objects.requireNonNull; + +public class ConnectorTableSchema +{ + private final SchemaTableName table; + private final List columns; + + public ConnectorTableSchema(SchemaTableName table, List columns) + { + requireNonNull(table, "table is null"); + requireNonNull(columns, "columns is null"); + + this.table = table; + this.columns = List.copyOf(columns); + } + + public SchemaTableName getTable() + { + return table; + } + + public List getColumns() + { + return columns; + } + + @Override + public String toString() + { + return new StringBuilder("ConnectorTableSchema{") + .append("table=").append(table) + .append(", columns=").append(columns) + .append('}') + .toString(); + } +} diff --git a/core/trino-spi/src/main/java/io/trino/spi/connector/SortOrder.java b/core/trino-spi/src/main/java/io/trino/spi/connector/SortOrder.java index 90d34053c330..ddfbf08a8e03 100644 --- a/core/trino-spi/src/main/java/io/trino/spi/connector/SortOrder.java +++ b/core/trino-spi/src/main/java/io/trino/spi/connector/SortOrder.java @@ -13,6 +13,8 @@ */ package io.trino.spi.connector; +import static java.lang.String.format; + public enum SortOrder { ASC_NULLS_FIRST(true, true), @@ -38,4 +40,12 @@ public boolean isNullsFirst() { return nullsFirst; } + + @Override + public String toString() + { + return format("%s %s", + ascending ? "ASC" : "DESC", + nullsFirst ? "NULLS FIRST" : "NULLS LAST"); + } } diff --git a/core/trino-spi/src/main/java/io/trino/spi/eventlistener/ColumnDetail.java b/core/trino-spi/src/main/java/io/trino/spi/eventlistener/ColumnDetail.java new file mode 100644 index 000000000000..8e4b2be7b70b --- /dev/null +++ b/core/trino-spi/src/main/java/io/trino/spi/eventlistener/ColumnDetail.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 io.trino.spi.eventlistener; + +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Objects; + +import static java.util.Objects.requireNonNull; + +public class ColumnDetail +{ + private final String catalog; + private final String schema; + private final String table; + private final String columnName; + + public ColumnDetail(String catalog, String schema, String table, String columnName) + { + this.catalog = requireNonNull(catalog, "catalog is null"); + this.schema = requireNonNull(schema, "schema is null"); + this.table = requireNonNull(table, "table is null"); + this.columnName = requireNonNull(columnName, "columnName is null"); + } + + @JsonProperty + public String getCatalog() + { + return catalog; + } + + @JsonProperty + public String getSchema() + { + return schema; + } + + @JsonProperty + public String getTable() + { + return table; + } + + @JsonProperty + public String getColumnName() + { + return columnName; + } + + @Override + public int hashCode() + { + return Objects.hash(catalog, schema, table, columnName); + } + + @Override + public boolean equals(Object obj) + { + if (obj == this) { + return true; + } + if ((obj == null) || (getClass() != obj.getClass())) { + return false; + } + ColumnDetail entry = (ColumnDetail) obj; + return Objects.equals(catalog, entry.catalog) && + Objects.equals(schema, entry.schema) && + Objects.equals(table, entry.table) && + Objects.equals(columnName, entry.columnName); + } +} diff --git a/core/trino-spi/src/main/java/io/trino/spi/eventlistener/OutputColumnMetadata.java b/core/trino-spi/src/main/java/io/trino/spi/eventlistener/OutputColumnMetadata.java new file mode 100644 index 000000000000..b7616e17e86c --- /dev/null +++ b/core/trino-spi/src/main/java/io/trino/spi/eventlistener/OutputColumnMetadata.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 io.trino.spi.eventlistener; + +import com.fasterxml.jackson.annotation.JsonCreator; + +import java.util.Objects; +import java.util.Set; + +import static java.util.Objects.requireNonNull; + +public class OutputColumnMetadata +{ + private final String columnName; + private final Set sourceColumns; + + @JsonCreator + public OutputColumnMetadata(String columnName, Set sourceColumns) + { + this.columnName = requireNonNull(columnName, "columnName is null"); + this.sourceColumns = requireNonNull(sourceColumns, "sourceColumns is null"); + } + + public String getColumnName() + { + return columnName; + } + + public Set getSourceColumns() + { + return sourceColumns; + } + + @Override + public int hashCode() + { + return Objects.hash(columnName, sourceColumns); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if ((obj == null) || (getClass() != obj.getClass())) { + return false; + } + OutputColumnMetadata other = (OutputColumnMetadata) obj; + return Objects.equals(columnName, other.columnName) && + Objects.equals(sourceColumns, other.sourceColumns); + } +} diff --git a/core/trino-spi/src/main/java/io/trino/spi/eventlistener/QueryOutputMetadata.java b/core/trino-spi/src/main/java/io/trino/spi/eventlistener/QueryOutputMetadata.java index 07f03188ac9c..a50545576d7c 100644 --- a/core/trino-spi/src/main/java/io/trino/spi/eventlistener/QueryOutputMetadata.java +++ b/core/trino-spi/src/main/java/io/trino/spi/eventlistener/QueryOutputMetadata.java @@ -15,6 +15,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; import java.util.Optional; import static java.util.Objects.requireNonNull; @@ -24,15 +25,17 @@ public class QueryOutputMetadata private final String catalogName; private final String schema; private final String table; + private final Optional> columns; private final Optional connectorOutputMetadata; private final Optional jsonLengthLimitExceeded; - public QueryOutputMetadata(String catalogName, String schema, String table, Optional connectorOutputMetadata, Optional jsonLengthLimitExceeded) + public QueryOutputMetadata(String catalogName, String schema, String table, Optional> columns, Optional connectorOutputMetadata, Optional jsonLengthLimitExceeded) { this.catalogName = requireNonNull(catalogName, "catalogName is null"); this.schema = requireNonNull(schema, "schema is null"); this.table = requireNonNull(table, "table is null"); + this.columns = requireNonNull(columns, "columns is null"); this.connectorOutputMetadata = requireNonNull(connectorOutputMetadata, "connectorOutputMetadata is null"); this.jsonLengthLimitExceeded = requireNonNull(jsonLengthLimitExceeded, "jsonLengthLimitExceeded is null"); } @@ -55,6 +58,12 @@ public String getTable() return table; } + @JsonProperty + public Optional> getColumns() + { + return columns; + } + @JsonProperty public Optional getConnectorOutputMetadata() { diff --git a/docs/pom.xml b/docs/pom.xml index 3afedf00df90..ac9902938b32 100644 --- a/docs/pom.xml +++ b/docs/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT trino-docs diff --git a/docs/src/main/sphinx/admin/dynamic-filtering.rst b/docs/src/main/sphinx/admin/dynamic-filtering.rst index b364506fa673..679be13ce3a2 100644 --- a/docs/src/main/sphinx/admin/dynamic-filtering.rst +++ b/docs/src/main/sphinx/admin/dynamic-filtering.rst @@ -23,6 +23,11 @@ from the processed dimension table on the right side of join. In the case of bro the runtime predicates generated from this collection are pushed into the local table scan on the left side of the join running on the same worker. +Dynamic filtering is enabled by default using the ``enable-dynamic-filtering`` +configuration property. To disable dynamic filtering, set the configuration +property to ``false``. Alternatively, use the session property +``enable_dynamic_filtering``. + Additionally, these runtime predicates are communicated to the coordinator over the network so that dynamic filtering can also be performed on the coordinator during enumeration of table scan splits. diff --git a/docs/src/main/sphinx/admin/properties-optimizer.rst b/docs/src/main/sphinx/admin/properties-optimizer.rst index 96cd1facec68..62be3c1c5ea3 100644 --- a/docs/src/main/sphinx/admin/properties-optimizer.rst +++ b/docs/src/main/sphinx/admin/properties-optimizer.rst @@ -125,7 +125,10 @@ join output rows can be skipped. Use connector provided table node partitioning when reading tables. For example, table node partitioning corresponds to Hive table buckets. -When set to ``true`` each table partition is read by a separate worker. +When set to ``true`` and minimal partition to task ratio is matched or exceeded, +each table partition is read by a separate worker. The minimal ratio is defined in +``optimizer.table-scan-node-partitioning-min-bucket-to-task-ratio``. + Partition reader assignments are distributed across workers for parallel processing. Use of table scan node partitioning can improve query performance by reducing query complexity. For example, diff --git a/docs/src/main/sphinx/admin/web-interface.rst b/docs/src/main/sphinx/admin/web-interface.rst index 088a22b49160..8fa1c62d2c4d 100644 --- a/docs/src/main/sphinx/admin/web-interface.rst +++ b/docs/src/main/sphinx/admin/web-interface.rst @@ -4,7 +4,7 @@ Web UI Trino provides a web-based user interface (UI) for monitoring a Trino cluster and managing queries. The Web UI is accessible on the coordinator via -HTTP/HTTPS, using the corresponding port number specified in the coordinator +HTTP or HTTPS, using the corresponding port number specified in the coordinator :ref:`config_properties`. It can be configured with :doc:`/admin/properties-web-interface`. The Web UI can be disabled entirely with the ``web-ui.enabled`` property. @@ -16,14 +16,10 @@ Authentication The Web UI requires users to authenticate. If Trino is not configured to require authentication, then any username can be used, and no password is required or -allowed. Typically, users should login with the same username that they use for +allowed. Typically, users login with the same username that they use for running queries. -Accessing the Web UI over HTTPS requires configuring an authentication type for -the Web UI or the Trino server. If no authentication type is configured for the -Web UI, then it will chosen based on the Trino server authentication type. - -If no system access control is installed, then all users will be able to view and kill +If no system access control is installed, then all users are able to view and kill any query. This can be restricted by using :ref:`query rules ` with the :doc:`/security/built-in-system-access-control`. Users always have permission to view or kill their own queries. @@ -31,12 +27,12 @@ or kill their own queries. Password authentication ^^^^^^^^^^^^^^^^^^^^^^^ -Typically, a :doc:`password authenticator ` +Typically, a password-based authentication method such as :doc:`LDAP ` or :doc:`password file ` is used to secure both the Trino server and the Web UI. When the Trino server is configured to use a password authenticator, the Web UI authentication type -is automatically set to ``form``. The Web UI will display a login form that accepts -a username and password. +is automatically set to ``form``. In this case, the Web UI displays a login form +that accepts a username and password. Fixed user authentication ^^^^^^^^^^^^^^^^^^^^^^^^^ diff --git a/docs/src/main/sphinx/connector/hive-s3.rst b/docs/src/main/sphinx/connector/hive-s3.rst index 5d98d89a40f1..f40a305d0630 100644 --- a/docs/src/main/sphinx/connector/hive-s3.rst +++ b/docs/src/main/sphinx/connector/hive-s3.rst @@ -76,7 +76,13 @@ Property Name Description the object has been created. ``hive.s3.upload-acl-type`` Canned ACL to use while uploading files to S3, defaults - to ``Private``. + to ``PRIVATE``. If the files are to be uploaded to an S3 + bucket owned by a different AWS user, the canned ACL has to be + set to one of the following: ``AUTHENTICATED_READ``, + ``AWS_EXEC_READ``, ``BUCKET_OWNER_FULL_CONTROL``, ``BUCKET_OWNER_READ``, + ``LOG_DELIVERY_WRITE``, ``PUBLIC_READ``, ``PUBLIC_READ_WRITE``. + Refer to the `AWS canned ACL `_ + guide to understand each option's definition. ``hive.s3.skip-glacier-objects`` Ignore Glacier objects rather than failing the query. This skips data that may be expected to be part of the table diff --git a/docs/src/main/sphinx/connector/hive.rst b/docs/src/main/sphinx/connector/hive.rst index 054bb0e94ed7..0bb7485bdba2 100644 --- a/docs/src/main/sphinx/connector/hive.rst +++ b/docs/src/main/sphinx/connector/hive.rst @@ -353,15 +353,9 @@ Property Name Description ``hive.file-status-cache-expire-time`` How long a cached directory listing should be considered ``1m`` valid. -``hive.parquet.time-zone`` Adjusts timestamp values to a specific time zone. JVM default - For Hive 3.1+, this should be set to UTC. - ``hive.rcfile.time-zone`` Adjusts binary encoded timestamp values to a specific JVM default time zone. For Hive 3.1+, this should be set to UTC. -``hive.orc.time-zone`` Sets the default time zone for legacy ORC files that did JVM default - not declare a time zone. - ``hive.timestamp-precision`` Specifies the precision to use for Hive columns of type ``MILLISECONDS`` ``timestamp``. Possible values are ``MILLISECONDS``, ``MICROSECONDS`` and ``NANOSECONDS``. Values with higher @@ -393,6 +387,48 @@ Property Name Description is limited to number of buckets. ================================================== ============================================================ ============ +ORC format configuration properties +----------------------------------- + +The following properties are used to configure the read and write operations +with ORC files performed by the Hive connector. + +.. list-table:: ORC format configuration properties + :widths: 30, 50, 20 + :header-rows: 1 + + * - ``hive.orc.time-zone`` + - Sets the default time zone for legacy ORC files that did not declare a + time zone. + - JVM default + * - ``hive.orc.use-columns-names`` + - Access ORC columns by name. By default, columns in ORC files are + accessed by their ordinal position in the Hive table definition. The + equivalent catalog session property is ``orc_use_column_names``. + - ``false`` + +Parquet format configuration properties +--------------------------------------- + +The following properties are used to configure the read and write operations +with Parquet files performed by the Hive connector. + +.. list-table:: Parquet format configuration properties + :widths: 30, 50, 20 + :header-rows: 1 + + * - ``hive.parquet.time-zone`` + - Adjusts timestamp values to a specific time zone. For Hive 3.1+, set + this to UTC. + - JVM default + * - ``hive.parquet.use-columns-names`` + - Access Parquet columns by name by default. Set this property to + ``false`` to access columns by their ordinal position in the Hive table + definition. The equivalent catalog session property is + ``parquet_use_column_names``. + - ``true`` + + Metastore configuration properties ---------------------------------- diff --git a/docs/src/main/sphinx/connector/iceberg.rst b/docs/src/main/sphinx/connector/iceberg.rst index 8d3d36152f87..ebd572c8e430 100644 --- a/docs/src/main/sphinx/connector/iceberg.rst +++ b/docs/src/main/sphinx/connector/iceberg.rst @@ -161,7 +161,7 @@ System tables and columns ------------------------- The connector supports queries of the table partitions. Given a table ``customer_accounts``, -``SELECT * customer_acccounts$partitions`` shows the table partitions, including the minimum +``SELECT * FROM customer_acccounts$partitions`` shows the table partitions, including the minimum and maximum values for the partition columns. Iceberg table properties diff --git a/docs/src/main/sphinx/connector/kafka.rst b/docs/src/main/sphinx/connector/kafka.rst index b0232e4f41e9..f8ed1d674fff 100644 --- a/docs/src/main/sphinx/connector/kafka.rst +++ b/docs/src/main/sphinx/connector/kafka.rst @@ -67,6 +67,15 @@ Property Name Description ``kafka.hide-internal-columns`` Controls whether internal columns are part of the table schema or not ``kafka.messages-per-split`` Number of messages that are processed by each Trino split, defaults to 100000 ``kafka.timestamp-upper-bound-force-push-down-enabled`` Controls if upper bound timestamp push down is enabled for topics using ``CreateTime`` mode +``kafka.security-protocol`` Security protocol for connection to Kafka cluster, defaults to ``PLAINTEXT`` +``kafka.ssl.keystore.location`` Location of the keystore file +``kafka.ssl.keystore.password`` Password for the keystore file +``kafka.ssl.keystore.type`` File format of the keystore file, defaults to ``JKS`` +``kafka.ssl.truststore.location`` Location of the truststore file +``kafka.ssl.truststore.password`` Password for the truststore file +``kafka.ssl.truststore.type`` File format of the truststore file, defaults to ``JKS`` +``kafka.ssl.key.password`` Password for the private key in the keystore file +``kafka.ssl.endpoint-identification-algorithm`` Endpoint identification algorithm used by clients to validate server host name, defaults to ``https`` ========================================================== ============================================================================== In addition, you need to configure :ref:`table schema and schema registry usage @@ -125,6 +134,73 @@ show up in ``DESCRIBE `` or ``SELECT *``. This property is optional; the default is ``true``. +``kafka.security-protocol`` +^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +Protocol used to communicate with brokers. +Valid values are: PLAINTEXT, SSL. + +This property is optional; default is ``PLAINTEXT``. + +``kafka.ssl.keystore.location`` +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +Location of the keystore file used for connection to Kafka cluster. + +This property is optional. + +``kafka.ssl.keystore.password`` +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +Password for the keystore file used for connection to Kafka cluster. + +This property is optional, but required when ``kafka.ssl.keystore.location`` is given. + +``kafka.ssl.keystore.type`` +^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +File format of the keystore file. +Valid values are: JKS, PKCS12. + +This property is optional; default is ``JKS``. + +``kafka.ssl.truststore.location`` +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +Location of the truststore file used for connection to Kafka cluster. + +This property is optional. + +``kafka.ssl.truststore.password`` +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +Password for the truststore file used for connection to Kafka cluster. + +This property is optional, but required when ``kafka.ssl.truststore.location`` is given. + +``kafka.ssl.truststore.type`` +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +File format of the truststore file. +Valid values are: JKS, PKCS12. + +This property is optional; default is ``JKS``. + +``kafka.ssl.key.password`` +^^^^^^^^^^^^^^^^^^^^^^^^^^ + +Password for the private key in the keystore file used for connection to Kafka cluster. + +This property is optional. This is required for clients only if two-way authentication is configured i.e. ``ssl.client.auth=required``. + +``kafka.ssl.endpoint-identification-algorithm`` +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +The endpoint identification algorithm used by clients to validate server host name for connection to Kafka cluster. +Kafka uses ``https`` as default. Use ``disabled`` to disable server host name validation. + +This property is optional; default is ``https``. + Internal columns ---------------- diff --git a/docs/src/main/sphinx/connector/oracle.rst b/docs/src/main/sphinx/connector/oracle.rst index 5481eb9fa1a3..c20c1c59685e 100644 --- a/docs/src/main/sphinx/connector/oracle.rst +++ b/docs/src/main/sphinx/connector/oracle.rst @@ -163,8 +163,6 @@ If an Oracle table uses a type not listed in the above table, then you can use t ``unsupported-type-handling`` configuration property to specify Trino behavior. For example: -- If ``unsupported-type-handling`` is set to ``FAIL``, then the - querying of an unsupported table fails. - If ``unsupported-type-handling`` is set to ``IGNORE``, then you can't see the unsupported types in Trino. - If ``unsupported-type-handling`` is set to ``CONVERT_TO_VARCHAR``, diff --git a/docs/src/main/sphinx/develop.rst b/docs/src/main/sphinx/develop.rst index 73e3dc0de5fe..1a254501fa29 100644 --- a/docs/src/main/sphinx/develop.rst +++ b/docs/src/main/sphinx/develop.rst @@ -18,3 +18,4 @@ This guide is intended for Trino contributors and plugin developers. develop/certificate-authenticator develop/group-provider develop/event-listener + develop/client-protocol diff --git a/docs/src/main/sphinx/develop/client-protocol.rst b/docs/src/main/sphinx/develop/client-protocol.rst new file mode 100644 index 000000000000..7ca48be162c2 --- /dev/null +++ b/docs/src/main/sphinx/develop/client-protocol.rst @@ -0,0 +1,265 @@ +====================== +Trino client REST API +====================== + +The REST API allows clients to submit SQL queries to Trino and receive the +results. Clients include the CLI, the JDBC driver, and others provided by +the community. The preferred method to interact with Trino is using these +existing clients. This document provides details about the API for reference. +It can also be used to implement your own client, if necessary. + +HTTP methods +------------ + +* A ``POST`` to ``/v1/statement`` runs the query string in the ``POST`` body, + and returns a JSON document containing the query results. If there are more + results, the JSON document contains a ``nextUri`` URL attribute. +* A ``GET`` to the ``nextUri`` attribute returns the next batch of query results. +* A ``DELETE`` to ``nextUri`` terminates a running query. + +Overview of query processing +---------------------------- + +A Trino client request is initiated by an HTTP ``POST`` to the endpoint +``/v1/statement``, with a ``POST`` body consisting of the SQL query string. +The caller may set HTTP header field ``X-Trino-User`` to the username for +the session. A number of other optional header fields are documented in +the following sections. + +If the client request returns an HTTP 503, that means the server was busy, +and the client should try again in 50-100 milliseconds. Any HTTP status other +than 503 or 200 means that query processing has failed. + +The ``/v1/statement`` ``POST`` request returns a JSON document of type +``QueryResults``, as well as a collection of response headers. The +``QueryResults`` document contains an ``error`` field of type +``QueryError`` if the query has failed, and if that object is not present, +the query succeeded. Important members of ``QueryResults`` are documented +in the following sections. + +If the ``data`` field of the JSON document is set, it contains a list of the +rows of data. The ``columns`` field is set to a list of the +names and types of the columns returned by the query. Most of the response +headers are treated like browser cookies by the client, and echoed back +as request headers in subsequent client requests, as documented below. + +If the JSON document returned by the ``POST`` to ``/v1/statement`` does not +contain a ``nextUri`` link, the query has completed, either successfully or +unsuccessfully, and no additional requests need to be made. If the +``nextUri`` link is present in the document, there are more query results +to be fetched. The client should loop executing a ``GET`` request +to the ``nextUri`` returned in the ``QueryResults`` response object until +``nextUri`` is absent from the response. + +The ``status`` field of the JSON document is for human consumption only, and +provides a hint about the query state. It can not be used to tell if the +query is finished. + +Important ``QueryResults`` attributes +------------------------------------- + +The most important attributes of the ``QueryResults`` JSON document returned by the REST API +endpoints are listed in this table. Refer to the class ``io.trino.client.QueryResults`` in +module ``trino-client`` for more details. + +.. list-table:: ``QueryResults attributes`` + :widths: 25, 55 + :header-rows: 1 + + * - Attribute + - Description + * - ``id`` + - The ID of the query. + * - ``nextUri`` + - If present, the URL to use for subsequent ``GET`` or + ``DELETE`` requests. If not present, the query is complete or + ended in error. + * - ``columns`` + - A list of the names and types of the columns returned by the query. + * - ``data`` + - The ``data`` attribute contains a list of the rows returned by the + query request. Each row is itself a list that holds values of the + columns in the row, in the order specified by the ``columns`` + attribute. + * - ``updateType`` + - A human-readable string representing the operation. For a + ``CREATE TABLE`` request, the ``updateType`` is + "CREATE TABLE"; for ``SET SESSION`` it is "SET SESSION"; etc. + * - ``error`` + - If query failed, the ``error`` attribute contains a + ``QueryError`` object. That object contains a ``message``, an + ``errorCode`` and other information about the error. See the + ``io.trino.client.QueryError`` class in module ``trino-client`` + for more details. + + +``QueryResults`` diagnostic attributes +-------------------------------------- + +These ``QueryResults`` data members may be useful in tracking down problems: + +.. list-table:: ``QueryResults diagnostic attributes`` + :widths: 20, 20, 40 + :header-rows: 1 + + * - Attribute + - Type + - Description + * - ``queryError`` + - ``QueryError`` + - Non-null only if the query resulted in an error. + * - ``failureInfo`` + - ``FailureInfo`` + - ``failureInfo`` has detail on the reason for the failure, including + a stack trace, and ``FailureInfo.errorLocation``, providing the + query line number and column number where the failure was detected. + * - ``warnings`` + - ``List`` + - A usually-empty list of warnings. + * - ``statementStats`` + - ``StatementStats`` + - A class containing statistics about the query execution. Of + particular interest is ``StatementStats.rootStage``, of type + ``StageStats``, providing statistics on the execution of each of + the stages of query processing. + +Client request headers +---------------------- + +This table lists all supported client request headers. Many of the +headers can be updated in the client as response headers, and supplied +in subsequent requests, just like browser cookies. + +.. list-table:: Client request headers + :widths: 30, 50 + :header-rows: 1 + + * - Header name + - Description + * - ``X-Trino-User`` + - Specifies the session user; must be supplied with every + request to ``/v1/statement``. + * - ``X-Trino-Source`` + - For reporting purposes, this supplies the name of the software + that submitted the query. + * - ``X-Trino-Catalog`` + - The catalog context for query processing. Set by response + header ``X-Trino-Set-Catalog``. + * - ``X-Trino-Schema`` + - The schema context for query processing. Set by response + header ``X-Trino-Set-Schema``. + * - ``X-Trino-Time-Zone`` + - The timezone for query processing. Defaults to the timezone + of the Trino cluster, and not the timezone of the client. + * - ``X-Trino-Language`` + - The language to use when processing the query and formatting + results, formatted as a Java ``Locale`` string, e.g., ``en-US`` + for US English. The language of the + session can be set on a per-query basis using the + ``X-Trino-Language`` HTTP header. + * - ``X-Trino-Trace-Token`` + - Supplies a trace token to the Trino engine to help identify + log lines that originate with this query request. + * - ``X-Trino-Session`` + - Supplies a comma-separated list of name=value pairs as session + properties. When the Trino client run a + ``SET SESSION name=value`` query, the name=value pair + is returned in the ``X-Set-Trino-Session`` response header, + and added to the client's list of session properties. + If the response header ``X-Trino-Clear-Session`` is returned, + its value is the name of a session property that is + removed from the client's accumulated list. + * - ``X-Trino-Role`` + - Sets the "role" for query processing. A "role" is represents + a collection of permissions. Set by response header + ``X-Trino-Set-Role``. See doc:/sql/create-role to + understand roles. + * - ``X-Trino-Prepared-Statement`` + - A comma-separated list of the name=value pairs, where the + names are names of previously prepared SQL statements, and + the values are keys that identify the executable form of the + named prepared statements. + * - ``X-Trino-Transaction-Id`` + - The transaction ID to use for query processing. Set + by response header ``X-Trino-Started-Transaction-Id`` and + cleared by ``X-Trino-Clear-Transaction-Id``. + * - ``X-Trino-Client-Info`` + - Contains arbitrary information about the client program + submitting the query. + * - ``X-Trino-Client-Tags`` + - A comma-separated list of "tag" strings, used to identify + Trino resource groups. + * - ``X-Trino-Resource-Estimate`` + - A comma-separated list of ``resource=value`` type + assigments. The possible choices of ``resource`` are + ``EXECUTION_TIME``, ``CPU_TIME``, ``PEAK_MEMORY`` and + ``PEAK_TASK_MEMORY``. ``EXECUTION_TIME`` and ``CPU_TIME`` + have values specified as airlift ``Duration`` strings + The format is a double precision number followed by + a ``TimeUnit`` string, e.g., of ``s`` for seconds, + ``m`` for minutes, ``h`` for hours, etc. "PEAK_MEMORY" and + "PEAK_TASK_MEMORY" are specified as as airlift ``DataSize`` strings, + whose format is an integer followed by ``B`` for bytes; ``kB`` for + kilobytes; ``mB`` for megabytes, ``gB`` for gigabytes, etc. + * - ``X-Trino-Extra-Credential`` + - Provides extra credentials to the connector. The header is + a name=value string that is saved in the session ``Identity`` + object. The name and value are only meaningful to the connector. + +Client response headers +----------------------- + +This table lists the supported client response headers. After receiving a +response, a client must update the request headers used in +subsequent requests to be consistent with the response headers received. + +.. list-table:: Client response headers + :widths: 30, 50 + :header-rows: 1 + + * - Header name + - Description + * - ``X-Trino-Set-Catalog`` + - Instructs the client to set the catalog in the + ``X-Trino-Catalog`` request header in subsequent client requests. + * - ``X-Trino-Set-Schema`` + - Instructs the client to set the schema in the + ``X-Trino-Schema`` request header in subsequent client requests. + * - ``X-Trino-Set-Session`` + - The value of the ``X-Trino-Set-Session`` response header is a + string of the form *property* = *value*. It + instructs the client include session property *property* with value + *value* in the ``X-Trino-Session`` header of subsequent + client requests. + * - ``X-Trino-Clear-Session`` + - Instructs the client to remove the session property with the + whose name is the value of the ``X-Trino-Clear-Session`` header + from the list of session properties + in the ``X-Trino-Session`` header in subsequent client requests. + * - ``X-Trino-Set-Role`` + - Instructs the client to set ``X-Trino-Role`` request header to the + catalog role supplied by the ``X-Trino-Set-Role`` header + in subsequent client requests. + * - ``X-Trino-Added-Prepare`` + - Instructs the client to add the name=value pair to the set of + prepared statements in the ``X-Trino-Prepared-Statements`` + request header in subsequent client requests. + * - ``X-Trino-Deallocated-Prepare`` + - Instructs the client to remove the prepared statement whose name + is the value of the ``X-Trino-Deallocated-Prepare`` header from + the client's list of prepared statements sent in the + ``X-Trino-Prepared-Statements`` request header in subsequent client + requests. + * - ``X-Trino-Started-Transaction-Id`` + - Provides the transaction ID that the client should pass back in the + ``X-Trino-Transaction-Id`` request header in subsequent requests. + * - ``X-Trino-Clear-Transaction-Id`` + - Instructs the client to clear the ``X-Trino-Transaction-Id`` request + header in subsequent requests. + +``ProtocolHeaders`` +------------------- + +Class ``io.trino.client.ProtocolHeaders``, in module ``trino-client``, +enumerates all the HTTP request and response headers allowed by the +Trino client REST API. diff --git a/docs/src/main/sphinx/develop/spi-overview.rst b/docs/src/main/sphinx/develop/spi-overview.rst index ea3b7557d202..02cdabda2210 100644 --- a/docs/src/main/sphinx/develop/spi-overview.rst +++ b/docs/src/main/sphinx/develop/spi-overview.rst @@ -90,7 +90,7 @@ plugin directory and add the relevant jars to that directory. By default, the plugin directory is the ``plugin`` directory relative to the directory in which Trino is installed, but it is configurable using the -configuration variable ``catalog.config-dir``. In order for Trino to pick up +configuration variable ``plugin.dir``. In order for Trino to pick up the new plugin, you must restart Trino. Plugins must be installed on all nodes in the Trino cluster (coordinator and workers). diff --git a/docs/src/main/sphinx/functions/datetime.rst b/docs/src/main/sphinx/functions/datetime.rst index 282f8269e55e..7c3731123aa2 100644 --- a/docs/src/main/sphinx/functions/datetime.rst +++ b/docs/src/main/sphinx/functions/datetime.rst @@ -131,9 +131,9 @@ Date and time functions Returns a timestamp with time zone from ``timestamp`` with precision ``p`` and ``zone``. -.. function:: from_unixtime(unixtime) -> timestamp(3) +.. function:: from_unixtime(unixtime) -> timestamp(3) with time zone - Returns the UNIX timestamp ``unixtime`` as a timestamp. ``unixtime`` is the + Returns the UNIX timestamp ``unixtime`` as a timestamp with time zone. ``unixtime`` is the number of seconds since ``1970-01-01 00:00:00 UTC``. .. function:: from_unixtime(unixtime, zone) -> timestamp(3) with time zone @@ -150,9 +150,9 @@ Date and time functions using ``hours`` and ``minutes`` for the time zone offset. ``unixtime`` is the number of seconds since ``1970-01-01 00:00:00`` in ``double`` data type. -.. function:: from_unixtime_nanos(unixtime) -> timestamp(9) +.. function:: from_unixtime_nanos(unixtime) -> timestamp(9) with time zone - Returns the UNIX timestamp ``unixtime`` as a timestamp. ``unixtime`` is the + Returns the UNIX timestamp ``unixtime`` as a timestamp with time zone. ``unixtime`` is the number of nanoseconds since ``1970-01-01 00:00:00.000000000 UTC``. .. data:: localtime diff --git a/docs/src/main/sphinx/functions/list.rst b/docs/src/main/sphinx/functions/list.rst index cf2af1b99862..22edc5f55c23 100644 --- a/docs/src/main/sphinx/functions/list.rst +++ b/docs/src/main/sphinx/functions/list.rst @@ -65,15 +65,19 @@ B - :func:`bing_tile_at` - :func:`bing_tile_coordinates` - :func:`bing_tile_polygon` +- ``bing_tile_quadkey`` - :func:`bing_tile_zoom_level` - :func:`bing_tiles_around` - :func:`bit_count` - :func:`bitwise_and` +- :func:`bitwise_and_agg` +- ``bitwise_left_shift`` - :func:`bitwise_not` - :func:`bitwise_or` -- :func:`bitwise_xor` -- :func:`bitwise_and_agg` - :func:`bitwise_or_agg` +- ``bitwise_right_shift`` +- ``bitwise_right_shift_arithmetic`` +- :func:`bitwise_xor` - :func:`bool_and` - :func:`bool_or` @@ -86,8 +90,8 @@ C - :func:`cbrt` - :func:`ceil` - :func:`ceiling` -- :func:`checksum` - :func:`char2hexint` +- :func:`checksum` - :func:`chr` - :func:`classify` - :ref:`coalesce ` @@ -105,8 +109,8 @@ C - :func:`cosine_similarity` - :func:`count` - :func:`count_if` -- :func:`covar_samp` - :func:`covar_pop` +- :func:`covar_samp` - :func:`crc32` - :func:`cume_dist` - :data:`current_date` @@ -140,6 +144,7 @@ E - :func:`e` - :func:`element_at` - :func:`empty_approx_set` +- ``evaluate_classifier_predictions`` - :func:`every` - :func:`extract` - :func:`exp` @@ -153,18 +158,22 @@ F - :func:`flatten` - :func:`floor` - :func:`format` +- ``format_datetime`` - :func:`from_base` - :func:`from_base64` - :func:`from_base64url` - :func:`from_big_endian_32` - :func:`from_big_endian_64` - :func:`from_encoded_polyline` +- ``from_geojson_geometry`` - :func:`from_hex` - :func:`from_ieee754_32` - :func:`from_ieee754_64` - :func:`from_iso8601_date` - :func:`from_iso8601_timestamp` +- ``from_iso8601_timestamp_nanos`` - :func:`from_unixtime` +- ``from_unixtime_nanos`` - :func:`from_utf8` G @@ -183,12 +192,14 @@ H - - :func:`hamming_distance` +- ``hash_counts`` - :func:`histogram` - :func:`hmac_md5` - :func:`hmac_sha1` - :func:`hmac_sha256` - :func:`hmac_sha512` - :func:`hour` +- ``human_readable_seconds`` I - @@ -196,6 +207,7 @@ I - :ref:`if ` - :func:`index` - :func:`infinity` +- ``intersection_cardinality`` - :func:`inverse_beta_cdf` - :func:`inverse_normal_cdf` - :func:`is_finite` @@ -210,6 +222,7 @@ I J - +- ``jaccard_index`` - :func:`json_array_contains` - :func:`json_array_get` - :func:`json_array_length` @@ -249,10 +262,12 @@ L - :func:`lower` - :func:`lpad` - :func:`ltrim` +- ``luhn_check`` M - +- ``make_set_digest`` - :func:`map` - :func:`map_agg` - :func:`map_concat` @@ -260,12 +275,14 @@ M - :func:`map_filter` - :func:`map_from_entries` - :func:`map_keys` +- ``map_union`` - :func:`map_values` - :func:`map_zip_with` - :func:`max` - :func:`max_by` - :func:`md5` - :func:`merge` +- ``merge_set_digest`` - :func:`millisecond` - :func:`min` - :func:`min_by` @@ -274,6 +291,7 @@ M - :func:`month` - :func:`multimap_agg` - :func:`multimap_from_entries` +- ``murmur3`` N - @@ -294,6 +312,8 @@ N O - +- ``objectid`` +- ``objectid_timestamp`` - :ref:`OR ` P @@ -355,9 +375,11 @@ S - :func:`sign` - :func:`simplify_geometry` - :func:`sin` -- :func:`slice` - :func:`skewness` +- :func:`slice` - :ref:`SOME ` +- ``spatial_partitioning`` +- ``spatial_partitions`` - :func:`split` - :func:`split_part` - :func:`split_to_map` @@ -385,8 +407,8 @@ S - :func:`ST_Equals` - :func:`ST_ExteriorRing` - :func:`ST_Geometries` -- :func:`ST_GeometryN` - :func:`ST_GeometryFromText` +- :func:`ST_GeometryN` - :func:`ST_GeometryType` - :func:`ST_GeomFromBinary` - :func:`ST_InteriorRingN` @@ -395,14 +417,15 @@ S - :func:`ST_Intersects` - :func:`ST_IsClosed` - :func:`ST_IsEmpty` -- :func:`ST_IsSimple` - :func:`ST_IsRing` +- :func:`ST_IsSimple` - :func:`ST_IsValid` - :func:`ST_Length` - :func:`ST_LineFromText` - :func:`ST_LineString` - :func:`ST_MultiPoint` - :func:`ST_NumGeometries` +- ``ST_NumInteriorRing`` - :func:`ST_NumPoints` - :func:`ST_Overlaps` - :func:`ST_Point` @@ -435,15 +458,18 @@ T - :func:`tan` - :func:`tanh` +- ``tdigest_agg`` - :func:`timezone_hour` - :func:`timezone_minute` - :func:`to_base` - :func:`to_base64` +- ``to_base64url`` - :func:`to_big_endian_32` - :func:`to_big_endian_64` - :func:`to_char` - :func:`to_date` - :func:`to_encoded_polyline` +- ``to_geojson_geometry`` - :func:`to_geometry` - :func:`to_hex` - :func:`to_ieee754_32` @@ -474,6 +500,7 @@ U - :func:`url_extract_host` - :func:`url_extract_parameter` - :func:`url_extract_path` +- ``url_extract_protocol`` - :func:`url_extract_port` - :func:`url_extract_query` - :func:`uuid` diff --git a/docs/src/main/sphinx/installation/cli.rst b/docs/src/main/sphinx/installation/cli.rst index 7a206199823c..32ba242085c9 100644 --- a/docs/src/main/sphinx/installation/cli.rst +++ b/docs/src/main/sphinx/installation/cli.rst @@ -25,6 +25,10 @@ make it executable with ``chmod +x``, then run it: Run the CLI with the ``--help`` option to see the available options. +The CLI uses the HTTP protocol and the +:doc:`Trino client REST API ` to communicate +with Trino. + Authentication -------------- diff --git a/docs/src/main/sphinx/installation/jdbc.rst b/docs/src/main/sphinx/installation/jdbc.rst index 9fa440a9c559..2b72d865fe2b 100644 --- a/docs/src/main/sphinx/installation/jdbc.rst +++ b/docs/src/main/sphinx/installation/jdbc.rst @@ -42,6 +42,10 @@ classpath, you'll typically need to restart your application in order to recognize the new driver. Then, depending on your application, you may need to manually register and configure the driver. +The CLI uses the HTTP protocol and the +:doc:`Trino client REST API ` to communicate +with Trino. + Registering and configuring the driver -------------------------------------- @@ -103,68 +107,73 @@ may not be specified using both methods. Parameter reference ------------------- -====================================== ======================================================================= -Name Description -====================================== ======================================================================= -``user`` Username to use for authentication and authorization. -``password`` Password to use for LDAP authentication. -``socksProxy`` SOCKS proxy host and port. Example: ``localhost:1080`` -``httpProxy`` HTTP proxy host and port. Example: ``localhost:8888`` -``clientInfo`` Extra information about the client. -``clientTags`` Client tags for selecting resource groups. Example: ``abc,xyz`` -``traceToken`` Trace token for correlating requests across systems. -``source`` Source name for the Trino query. This parameter should be used in - preference to ``ApplicationName``. Thus, it takes precedence - over ``ApplicationName`` and/or ``applicationNamePrefix``. -``applicationNamePrefix`` Prefix to append to any specified ``ApplicationName`` client info - property, which is used to set the source name for the Trino query - if the ``source`` parameter has not been set. If neither this - property nor ``ApplicationName`` or ``source`` are set, the source - name for the query is ``trino-jdbc``. -``accessToken`` Access token for token based authentication. -``SSL`` Use HTTPS for connections -``SSLVerification`` The method of SSL verification. There are three modes: ``FULL`` - (default), ``CA`` and ``NONE``. For ``FULL``, the normal TLS - verification is performed. For ``CA``, only the CA is verified but - hostname mismatch is allowed. For ``NONE``, there is no verification. -``SSLKeyStorePath`` The location of the Java KeyStore file that contains the certificate - and private key to use for authentication. -``SSLKeyStorePassword`` The password for the KeyStore. -``SSLKeyStoreType`` The type of the KeyStore. The default type is provided by the Java - ``keystore.type`` security property or ``jks`` if none exists. -``SSLTrustStorePath`` The location of the Java TrustStore file to use. - to validate HTTPS server certificates. -``SSLTrustStorePassword`` The password for the TrustStore. -``SSLTrustStoreType`` The type of the TrustStore. The default type is provided by the Java - ``keystore.type`` security property or ``jks`` if none exists. -``KerberosRemoteServiceName`` Trino coordinator Kerberos service name. This parameter is - required for Kerberos authentication. -``KerberosPrincipal`` The principal to use when authenticating to the Trino coordinator. -``KerberosUseCanonicalHostname`` Use the canonical hostname of the Trino coordinator for the Kerberos - service principal by first resolving the hostname to an IP address - and then doing a reverse DNS lookup for that IP address. - This is enabled by default. -``KerberosServicePrincipalPattern`` Trino coordinator Kerberos service principal pattern. The default is - ``${SERVICE}@${HOST}``. ``${SERVICE}`` is replaced with the value of - ``KerberosRemoteServiceName`` and ``${HOST}`` is replaced with the - hostname of the coordinator (after canonicalization if enabled). -``KerberosConfigPath`` Kerberos configuration file. -``KerberosKeytabPath`` Kerberos keytab file. -``KerberosCredentialCachePath`` Kerberos credential cache. -``extraCredentials`` Extra credentials for connecting to external services, - specified as a list of key-value pairs. For example, - ``foo:bar;abc:xyz`` creates the credential named ``abc`` - with value ``xyz`` and the credential named ``foo`` with value ``bar``. -``roles`` Authorization roles to use for catalogs, specified as a list of - key-value pairs for the catalog and role. For example, - ``catalog1:roleA;catalog2:roleB`` sets ``roleA`` - for ``catalog1`` and ``roleB`` for ``catalog2``. -``sessionProperties`` Session properties to set for the system and for catalogs, - specified as a list of key-value pairs. - For example, ``abc:xyz;example.foo:bar`` sets the system property - ``abc`` to the value ``xyz`` and the ``foo`` property for - catalog ``example`` to the value ``bar``. -``externalAuthentication`` Use a local web browser to authenticate with an identity provider (IdP) - that has been configured for the Trino coordinator. - See :doc:`/security/oauth2` for more details. -====================================== ======================================================================= +============================================================ ======================================================================= +Name Description +============================================================ ======================================================================= +``user`` Username to use for authentication and authorization. +``password`` Password to use for LDAP authentication. +``socksProxy`` SOCKS proxy host and port. Example: ``localhost:1080`` +``httpProxy`` HTTP proxy host and port. Example: ``localhost:8888`` +``clientInfo`` Extra information about the client. +``clientTags`` Client tags for selecting resource groups. Example: ``abc,xyz`` +``traceToken`` Trace token for correlating requests across systems. +``source`` Source name for the Trino query. This parameter should be used in + preference to ``ApplicationName``. Thus, it takes precedence + over ``ApplicationName`` and/or ``applicationNamePrefix``. +``applicationNamePrefix`` Prefix to append to any specified ``ApplicationName`` client info + property, which is used to set the source name for the Trino query + if the ``source`` parameter has not been set. If neither this + property nor ``ApplicationName`` or ``source`` are set, the source + name for the query is ``trino-jdbc``. +``accessToken`` Access token for token based authentication. +``SSL`` Use HTTPS for connections +``SSLVerification`` The method of SSL verification. There are three modes: ``FULL`` + (default), ``CA`` and ``NONE``. For ``FULL``, the normal TLS + verification is performed. For ``CA``, only the CA is verified but + hostname mismatch is allowed. For ``NONE``, there is no verification. +``SSLKeyStorePath`` The location of the Java KeyStore file that contains the certificate + and private key to use for authentication. +``SSLKeyStorePassword`` The password for the KeyStore. +``SSLKeyStoreType`` The type of the KeyStore. The default type is provided by the Java + ``keystore.type`` security property or ``jks`` if none exists. +``SSLTrustStorePath`` The location of the Java TrustStore file to use. + to validate HTTPS server certificates. +``SSLTrustStorePassword`` The password for the TrustStore. +``SSLTrustStoreType`` The type of the TrustStore. The default type is provided by the Java + ``keystore.type`` security property or ``jks`` if none exists. +``KerberosRemoteServiceName`` Trino coordinator Kerberos service name. This parameter is + required for Kerberos authentication. +``KerberosPrincipal`` The principal to use when authenticating to the Trino coordinator. +``KerberosUseCanonicalHostname`` Use the canonical hostname of the Trino coordinator for the Kerberos + service principal by first resolving the hostname to an IP address + and then doing a reverse DNS lookup for that IP address. + This is enabled by default. +``KerberosServicePrincipalPattern`` Trino coordinator Kerberos service principal pattern. The default is + ``${SERVICE}@${HOST}``. ``${SERVICE}`` is replaced with the value of + ``KerberosRemoteServiceName`` and ``${HOST}`` is replaced with the + hostname of the coordinator (after canonicalization if enabled). +``KerberosConfigPath`` Kerberos configuration file. +``KerberosKeytabPath`` Kerberos keytab file. +``KerberosCredentialCachePath`` Kerberos credential cache. +``extraCredentials`` Extra credentials for connecting to external services, + specified as a list of key-value pairs. For example, + ``foo:bar;abc:xyz`` creates the credential named ``abc`` + with value ``xyz`` and the credential named ``foo`` with value ``bar``. +``roles`` Authorization roles to use for catalogs, specified as a list of + key-value pairs for the catalog and role. For example, + ``catalog1:roleA;catalog2:roleB`` sets ``roleA`` + for ``catalog1`` and ``roleB`` for ``catalog2``. +``sessionProperties`` Session properties to set for the system and for catalogs, + specified as a list of key-value pairs. + For example, ``abc:xyz;example.foo:bar`` sets the system property + ``abc`` to the value ``xyz`` and the ``foo`` property for + catalog ``example`` to the value ``bar``. +``externalAuthentication`` Use a local web browser to authenticate with an identity provider (IdP) + that has been configured for the Trino coordinator. + See :doc:`/security/oauth2` for more details. +``disableCompression`` Whether compression should be enabled. +``assumeLiteralNamesInMetadataCallsForNonConformingClients`` When enabled, the name patterns passed to ``DatabaseMetaData`` methods + are treated as literals. You can use this as a workaround for + applications that do not escape schema or table names when passing them + to ``DatabaseMetaData`` methods as schema or table name patterns. +============================================================ ======================================================================= diff --git a/docs/src/main/sphinx/release.rst b/docs/src/main/sphinx/release.rst index 44ed32e96bce..31374ad99e6a 100644 --- a/docs/src/main/sphinx/release.rst +++ b/docs/src/main/sphinx/release.rst @@ -10,6 +10,7 @@ Release notes .. toctree:: :maxdepth: 1 + release/release-355 release/release-354 release/release-353 release/release-352 diff --git a/docs/src/main/sphinx/release/release-355.md b/docs/src/main/sphinx/release/release-355.md new file mode 100644 index 000000000000..26d3f939ba5d --- /dev/null +++ b/docs/src/main/sphinx/release/release-355.md @@ -0,0 +1,52 @@ +# Release 355 (8 Apr 2021) + +## General + +* Report tables that are directly referenced by a query in `QueryCompletedEvent`. ({issue}`7330`) +* Report columns that are the target of `INSERT` or `UPDATE` queries in `QueryCompletedEvent`. This includes + information about which input columns they are derived from. ({issue}`7425`, {issue}`7465`) +* Rename `optimizer.plan-with-table-node-partitioning` config property to `optimizer.use-table-scan-node-partitioning`. ({issue}`7257`) +* Improve query parallelism when table bucket count is small compared to number of nodes. + This optimization is now triggered automatically when the ratio between table buckets and + possible table scan tasks exceeds or is equal to `optimizer.table-scan-node-partitioning-min-bucket-to-task-ratio`. ({issue}`7257`) +* Include information about {doc}`/admin/spill` in {doc}`/sql/explain-analyze`. ({issue}`7427`) +* Disallow inserting data into tables that have row filters. ({issue}`7346`) +* Improve performance of queries that can benefit from both {doc}`/optimizer/cost-based-optimizations` and join pushdown + by giving precedence to cost-based optimizations. ({issue}`7331`) +* Fix inconsistent behavior for {func}`to_unixtime` with values of type `timestamp(p)`. ({issue}`7450`) +* Change return type of {func}`from_unixtime` and {func}`from_unixtime_nanos` to `timestamp(p) with time zone`. ({issue}`7460`) + +## Security + +* Add support for configuring multiple password authentication plugins. ({issue}`7151`) + +## JDBC driver + +* Add `assumeLiteralNamesInMetadataCallsForNonConformingClients` parameter for use as a workaround when + applications do not properly escape schema or table names in calls to `DatabaseMetaData` methods. ({issue}`7438`) + +## ClickHouse connector + +* Support creating tables with MergeTree storage engine. ({issue}`7135`) + +## Hive connector + +* Support Hive views containing `LATERAL VIEW json_tuple(...) AS ...` syntax. ({issue}`7242`) +* Fix incorrect results when reading from a Hive view that uses array subscript operators. ({issue}`7271`) +* Fix incorrect results when querying the `$file_modified_time` hidden column. ({issue}`7511`) + +## Phoenix connector + +* Improve performance when fetching table metadata during query analysis. ({issue}`6975`) +* Improve performance of queries with `ORDER BY ... LIMIT` clause when the computation + can be pushed down to the underlying database. ({issue}`7490`) + +## SQL Server connector + +* Improve performance when fetching table metadata during query analysis. ({issue}`6975`) + +## SPI + +* Engine now uses `ConnectorMaterializedViewDefinition#storageTable` + to determine materialized view storage table. ({issue}`7319`) + diff --git a/docs/src/main/sphinx/security.rst b/docs/src/main/sphinx/security.rst index 12693d40030f..09a73e04dea9 100644 --- a/docs/src/main/sphinx/security.rst +++ b/docs/src/main/sphinx/security.rst @@ -2,6 +2,14 @@ Security ******** +Introduction +============ + +.. toctree:: + :maxdepth: 1 + + security/overview + Cluster access security ======================= @@ -9,6 +17,9 @@ Cluster access security :maxdepth: 1 security/tls + security/inspect-pem + security/inspect-jks + security/authentication-types security/ldap security/password-file security/salesforce diff --git a/docs/src/main/sphinx/security/authentication-types.rst b/docs/src/main/sphinx/security/authentication-types.rst new file mode 100644 index 000000000000..f548d4d9bb66 --- /dev/null +++ b/docs/src/main/sphinx/security/authentication-types.rst @@ -0,0 +1,64 @@ +==================== +Authentication types +==================== + +Trino supports multiple authentication types to ensure all users of the system +are authenticated. Different authenticators allow user management in one or more +systems. Using :doc:`TLS ` is required for all authentications types. + +You can configure one or more authentication types with the +``http-server.authentication.type`` property. The following authentication types +and authenticators are available: + +* ``PASSSWORD`` for :doc:`password-file`, :doc:`ldap`, and :doc:`salesforce` +* ``OAUTH2`` for :doc:`oauth2` +* ``CERTIFICATE`` for certificate authentication +* ``JWT`` for Java Web Token (JWT) authentication +* ``KERBEROS`` for :doc:`Kerberos authentication ` + +Get started with a basic password authentication configuration backed by a +:doc:`password file `: + +.. code-block:: properties + + http-server.authentication.type=PASSWORD + + +Multiple authentication types +----------------------------- + +You can use multiple authentication types, separated with commas in the +configuration: + +.. code-block:: properties + + http-server.authentication.type=PASSWORD,CERTIFICATE + + +Authentication is performed in order of the entries, and first successful +authentication results in access, using the :doc:`mapped user ` +from that authentication method. + +Multiple password authenticators +-------------------------------- + +You can use multiple password authenticator types by referencing multiple +configuration files: + +.. code-block:: properties + + http-server.authentication.type=PASSWORD + password-authenticator.config-files=etc/ldap1.properties,etc/ldap2.properties,etc/password.properties + +In the preceding example, the configuration files ``ldap1.properties`` and +``ldap1.properties`` are regular :doc:`LDAP authenticator configuration files +`. The ``password.properties`` is a :doc:`password file authenticator +configuration file `. + +Relative paths to the installation directory or absolute paths can be used. + +User authentication credentials are first validated against the LDAP server from +``ldap1``, then the separate server from ``ldap2``, and finally the password +file. First successful authentication results in access, and no further +authenticators are called. + diff --git a/docs/src/main/sphinx/security/cli.rst b/docs/src/main/sphinx/security/cli.rst index f33019c155dc..cb804af2a0f3 100644 --- a/docs/src/main/sphinx/security/cli.rst +++ b/docs/src/main/sphinx/security/cli.rst @@ -33,13 +33,12 @@ principal. .. include:: ktadd-note.fragment -Java keystore file for TLS -^^^^^^^^^^^^^^^^^^^^^^^^^^ +Configuration for TLS +^^^^^^^^^^^^^^^^^^^^^ -Access to the Trino coordinator must be through HTTPS when using Kerberos -authentication. The Trino coordinator uses a :ref:`Java Keystore -` file for its TLS configuration. This file can be -copied to the client machine and used for its configuration. +When using Kerberos authentication, access to the Trino coordinator must be +through HTTPS. If you have not yet configured HTTPS/TLS for your coordinator, +refer to :doc:`HTTPS and TLS `. Trino CLI execution -------------------- diff --git a/docs/src/main/sphinx/security/inspect-jks.rst b/docs/src/main/sphinx/security/inspect-jks.rst new file mode 100644 index 000000000000..0de6677ddc76 --- /dev/null +++ b/docs/src/main/sphinx/security/inspect-jks.rst @@ -0,0 +1,132 @@ +========= +JKS files +========= + +This topic describes how to validate a :ref:`Java keystore (JKS) ` +file used to configure :doc:`/security/tls`. + +The Java KeyStore (JKS) system is provided as part of your Java installation. +Private keys and certificates for your server are stored in a *keystore* file. +The JKS system supports both PKCS #12 ``.p12`` files as well as legacy +keystore ``.jks`` files. + +The keystore file itself is always password-protected. The keystore file can +have more than one key in the the same file, each addressed by its **alias** +name. + +If you receive a keystore file from your site's network admin group, verify that +it shows the correct information for your Trino cluster, as described next. + +.. _troubleshooting_keystore: + +Inspect and validate keystore +----------------------------- + +Inspect the keystore file to make sure it contains the correct information for +your Trino server. Use the ``keytool`` command, which is installed as part of +your Java installation, to retrieve information from your keystore file: + +.. code-block:: text + + keytool -list -v -keystore yourKeystore.jks + +Keystores always require a password. If not provided on the ``keytool`` command +line, ``keytool`` prompts for the password. + +Independent of the keystore's password, it is possible that an individual key +has its own password. It is easiest to make sure these passwords are the same. +If the JKS key inside the keystore has a different password, you are prompted +twice. + +In the output of the ``keytool -list`` command, look for: + +* The keystore may contain either a private key (``Entry type: + PrivateKeyEntry``) or certificate (``Entry type: trustedCertEntry``) or both. +* Modern browsers now enforce 398 days as the maximum validity period for a + certificate. Look for the ``Valid from ... until`` entry, and make sure the + time span does not exceed 398 days. +* Modern browsers and clients require the **SubjectAlternativeName** (SAN) + field. Make sure this shows the DNS name of your server, such as + ``DNS:cluster.example.com``. Certificates without SANs are not + supported. + + Example: + +.. code-block:: text + + SubjectAlternativeName [ + DNSName: cluster.example.com + ] + +If your keystore shows valid information for your cluster, proceed to configure +the Trino server, as described in :ref:`cert-placement` and +:ref:`configure-https`. + +The rest of this page describes additional steps that may apply in certain +circumstances. + +.. _import_to_keystore: + +Extra: add PEM to keystore +-------------------------- + +Your site may have standardized on using JKS semantics for all servers. If a +vendor sends you a PEM-encoded certificate file for your Trino server, you can +import it into a keystore with a command like the following. Consult ``keytool`` +references for different options. + +.. code-block:: shell + + keytool -trustcacerts -import -alias cluster -file localhost.pem -keystore localkeys.jks + +If the specified keystore file exists, ``keytool`` prompts for its password. If +you are creating a new keystore, ``keytool`` prompts for a new password, then +prompts you to confirm the same password. ``keytool`` shows you the +contents of the key being added, similar to the ``keytool -list`` format, then +prompts: + +.. code-block:: text + + Trust this certificate? [no]: + +Type ``yes`` to add the PEM certificate to the keystore. + +The ``alias`` name is an arbitrary string used as a handle for the certificate +you are adding. A keystore can contain multiple keys and certs, so ``keytool`` +uses the alias to address individual entries. + +.. _cli_java_truststore: + +Extra: Java truststores +----------------------- + +.. note:: + + Remember that there may be no need to identify a local truststore when + directly using a signed PEM-encoded certificate, independent of a keystore. + PEM certs can contain the server's private key and the certificate chain all + the way back to a recognzied CA. + +Truststore files contain a list of :ref:`Certificate Authorities ` +trusted by Java to validate the private keys of servers, plus a list of the +certificates of trusted TLS servers. The standard Java-provided truststore file, +``cacerts``, is part of your Java installation in a standard location. + +Keystores normally rely on the default location of the system truststore, which +therefore does not need to be configured. + +However, there are cases in which you need to use an alternate truststore. For +example, if your site relies on the JKS system, your network managers may have +appended site-specific, local CAs to the standard list, to validate locally +signed keys. + +If your server must use a custom truststore, identify its location in the +server's config properties file. For example: + +.. code-block:: text + + http-server.https.truststore.path=/mnt/shared/certs/localcacerts + http-server.https.truststore.key= + +If connecting clients such as browsers or the Trino CLI must be separately +configured, contact your site's network administrators for assistance. \ No newline at end of file diff --git a/docs/src/main/sphinx/security/inspect-pem.rst b/docs/src/main/sphinx/security/inspect-pem.rst new file mode 100644 index 000000000000..faa74a7ea32a --- /dev/null +++ b/docs/src/main/sphinx/security/inspect-pem.rst @@ -0,0 +1,130 @@ +========= +PEM files +========= + +PEM (Privacy Enhanced Mail) is a standard for public key and certificate +information, and an encoding standard used to transmit keys and certificates. + +Trino supports PEM-encoded certificates. If you want to use other supported +formats, see: + +* :doc:`JKS keystores ` +* :ref:`PKCS 12 ` stores. (Look up alternate commands for these in + ``openssl`` references.) + +A single PEM-encoded file can contain either certificate or key pair +information, or both in the same file. Certified keys can contain a chain of +certificates from successive certificate authorities. + +Follow the steps in this topic to inspect and validate PEM-encoded key and +certificate files. See :ref:`troubleshooting_keystore` to validate JKS +keystores. + +.. _inspect_pems: + +Inspect PEM file +---------------- + +The file name extensions shown on this page are examples only; there is no +extension naming standard. + +You may receive a single file that includes a private key and its certificate, +or separate files. If you received separate files, concatenate them into one, +typically in order from key to certificate. For example: + +.. code-block:: shell + + cat clustercoord.key clustercoord.cert > clustercoord.pem + +Next, use the ``cat`` command to view this plain text file. For example: + +.. code-block:: shell + + cat clustercoord.pem | less + +Make sure the PEM file shows at least one ``KEY`` and one ``CERTIFICATE`` +section. A key section looks something like the following: + +.. code-block:: text + + -----BEGIN PRIVATE KEY----- + MIIEowIBAAKCAQEAwJL8CLeDFAHhZe3QOOF1vWt4Vuk9vyO38Y1y9SgBfB02b2jW + .... + -----END PRIVATE KEY----- + +If your key section reports ``BEGIN ENCRYPTED PRIVATE KEY`` instead, this means +the key is encrypted and you must use the password to open or inspect the key. +You may have specified the password when requesting the key, or the password +could be assigned by your site's network managers. + +If your key section reports ``BEGIN EC PRIVATE KEY`` or ``BEGIN DSA PRIVATE +KEY``, this designates a key using Elliptical Curve or DSA alternatives to RSA. + +The certificate section looks like the following example: + +.. code-block:: text + + -----BEGIN CERTIFICATE----- + MIIDujCCAqICAQEwDQYJKoZIhvcNAQEFBQAwgaIxCzAJBgNVBAYTAlVTMRYwFAYD + .... + -----END CERTIFICATE----- + -----BEGIN CERTIFICATE----- + MIIDwjCCAqoCCQCxyqwZ9GK50jANBgkqhkiG9w0BAQsFADCBojELMAkGA1UEBhMC + .... + -----END CERTIFICATE----- + +The file can show a single certificate section, or more than one to express a +chain of authorities, each certifying the previous. + +.. _validate_pems: + +Validate PEM key section +------------------------ + +This page presumes your system provides the ``openssl`` command from OpenSSL 1.1 +or later. + +Test an RSA private key's validity with the following command: + +.. code-block:: text + + openssl rsa -in clustercoord.pem -check -noout + +Look for the following confirmation message: + +.. code-block:: text + + RSA key ok + +.. note:: + + Consult ``openssl`` references for the appropriate versions of the + verification commands for EC or DSA keys. + +Validate PEM certificate section +-------------------------------- + +Analyze the certificate section of your PEM file with the following ``openssl`` +command: + +.. code-block:: text + + openssl x509 -in clustercoord.pem -text -noout + +If your certificate was generated with a password, ``openssl`` prompts for it. + +In the output of the ``openssl`` command, look for the following +characteristics: + +* Modern browsers now enforce 398 days as the maximum validity period for a + certificate. Look for ``Not Before`` and ``Not After`` dates in the + ``Validity`` section of the output, and make sure the time span does not + exceed 398 days. +* Modern browsers and clients require the **Subject Alternative Name** (SAN) + field. Make sure this shows the DNS name of your server, such as + ``DNS:clustercoord.example.com``. Certificates without SANs are not + supported. + +If your PEM certificate shows valid information for your cluster, proceed to +configure the server, as described in :ref:`cert-placement` and +:ref:`configure-https`. diff --git a/docs/src/main/sphinx/security/ldap.rst b/docs/src/main/sphinx/security/ldap.rst index fd0268da3d7f..c73f96010e50 100644 --- a/docs/src/main/sphinx/security/ldap.rst +++ b/docs/src/main/sphinx/security/ldap.rst @@ -21,8 +21,8 @@ Trino server configuration Trino coordinator node configuration ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ -Access to the Trino coordinator should be through HTTPS. You can do that -by creating a :ref:`server_java_keystore` on the coordinator. +Access to the Trino coordinator should be through HTTPS, configured as described +on :doc:`HTTPS and TLS `. You also need to make changes to the Trino configuration files. LDAP authentication is configured on the coordinator in two parts. @@ -49,16 +49,17 @@ to the coordinator's ``config.properties`` file: ============================================================= ====================================================== Property Description ============================================================= ====================================================== -``http-server.authentication.type`` Enable password authentication for the Trino - coordinator. Must be set to ``PASSWORD``. +``http-server.authentication.type`` Enable the password :doc:`authentication type ` + for the Trino coordinator. Must be set to ``PASSWORD``. ``http-server.https.enabled`` Enables HTTPS access for the Trino coordinator. Should be set to ``true``. Default value is ``false``. ``http-server.https.port`` HTTPS server port. -``http-server.https.keystore.path`` The location of the Java Keystore file that will be - used to secure TLS. -``http-server.https.keystore.key`` The password for the keystore. This must match the - password you specified when creating the keystore. +``http-server.https.keystore.path`` The location of the PEM or Java keystore file + is used to enable TLS. +``http-server.https.keystore.key`` The password for the PEM or Java keystore. This + must match the password you specified when creating + the PEM or keystore. ``http-server.process-forwarded`` Enable treating forwarded HTTPS requests over HTTP as secure. Requires the ``X-Forwarded-Proto`` header to be set to ``https`` on forwarded requests. @@ -88,11 +89,11 @@ Property Description ================================== ====================================================== ``ldap.url`` The URL to the LDAP server. The URL scheme must be ``ldap://`` or ``ldaps://``. Connecting to the LDAP - server without SSL enabled requires + server without TLS enabled requires ``ldap.allow-insecure=true``. ``ldap.allow-insecure`` Allow using an LDAP connection that is not secured with TLS. -``ldap.ssl-trust-certificate`` The path to the PEM encoded trust certificate for the +``ldap.ssl-trust-certificate`` The path to the PEM encoded trust certificate for the LDAP server. This file should contain the LDAP server's certificate or its certificate authority. ``ldap.user-bind-pattern`` This property can be used to specify the LDAP user @@ -243,15 +244,8 @@ Environment configuration TLS configuration ~~~~~~~~~~~~~~~~~ -Access to the Trino coordinator should be through HTTPS when using LDAP -authentication. The Trino CLI can use either a :ref:`Java Keystore -` file or :ref:`Java Truststore ` -for its TLS configuration. - -If you are using a keystore file, it can be copied to the client machine and used -for its TLS configuration. If you are using truststore, you can either use -default Java truststores or create a custom truststore on the CLI. We do not -recommend using self-signed certificates in production. +When using LDAP authentication, access to the Trino coordinator must be through +:doc:`HTTPS/TLS `. Trino CLI execution ^^^^^^^^^^^^^^^^^^^^ @@ -289,7 +283,7 @@ Option Description to secure TLS. ``--keystore-password`` The password for the keystore. This must match the password you specified when creating the keystore. -``--truststore-path`` The location of the Java Truststore file that will be used +``--truststore-path`` The location of the Java truststore file that will be used to secure TLS. ``--truststore-password`` The password for the truststore. This must match the password you specified when creating the truststore. @@ -319,10 +313,10 @@ you can change the :ref:`log level ` for the LDAP authenticator: io.trino.plugin.password=DEBUG -SSL debugging for Trino CLI +TLS debugging for Trino CLI ^^^^^^^^^^^^^^^^^^^^^^^^^^^^ -If you encounter any SSL related errors when running the Trino CLI, you can run +If you encounter any TLS related errors when running the Trino CLI, you can run the CLI using the ``-Djavax.net.debug=ssl`` parameter for debugging. Use the Trino CLI executable JAR to enable this. For example: @@ -334,30 +328,30 @@ Trino CLI executable JAR to enable this. For example: --server https://coordinator:8443 \ -Common SSL errors -~~~~~~~~~~~~~~~~~ +Common TLS/SSL errors +~~~~~~~~~~~~~~~~~~~~~ java.security.cert.CertificateException: No subject alternative names present ***************************************************************************** This error is seen when the Trino coordinator’s certificate is invalid, and does not have the IP you provide -in the ``--server`` argument of the CLI. You have to regenerate the coordinator's SSL certificate +in the ``--server`` argument of the CLI. You have to regenerate the coordinator's TLS certificate with the appropriate :abbr:`SAN (Subject Alternative Name)` added. Adding a SAN to this certificate is required in cases where ``https://`` uses IP address in the URL, rather than the domain contained in the coordinator's certificate, and the certificate does not contain the :abbr:`SAN (Subject Alternative Name)` parameter with the matching IP address as an alternative attribute. -Authentication or SSL errors with JDK upgrade +Authentication or TLS errors with JDK upgrade ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ Starting with the JDK 8u181 release, to improve the robustness of LDAPS -(secure LDAP over TLS) connections, endpoint identification algorithms have -been enabled by default. See release notes +(secure LDAP over TLS) connections, endpoint identification algorithms were +enabled by default. See release notes `from Oracle `_. The same LDAP server certificate on the Trino coordinator, running on JDK version >= 8u181, that was previously able to successfully connect to an -LDAPS server, may now fail with the below error: +LDAPS server, may now fail with the following error: .. code-block:: text diff --git a/docs/src/main/sphinx/security/overview.rst b/docs/src/main/sphinx/security/overview.rst new file mode 100644 index 000000000000..a14f55b10440 --- /dev/null +++ b/docs/src/main/sphinx/security/overview.rst @@ -0,0 +1,149 @@ +================= +Security overview +================= + +After the initial :doc:`installation ` of your cluster, security +is the next major concern for successfully operating Trino. This overview +provides an introduction to different aspects of configuring security for your +Trino cluster. + +Aspects of configuring security +------------------------------- + +The default installation of Trino has no security features enabled. Security +can be enabled for different parts of the Trino architecture: + +* :ref:`security-client` +* :ref:`security-inside-cluster` +* :ref:`security-data-sources` + +Suggested configuration workflow +-------------------------------- + +To configure security for a new Trino cluster, follow this best practice +order of steps. Do not skip or combine steps. + +#. **Enable** :doc:`HTTPS/TLS ` + + * Work with your security team. + * Use a :ref:`load balancer or proxy ` to terminate + HTTPS, if possible. + * Use a globally trusted TLS certificate. + +#. **Enable authentication** + + * Start with :doc:`password file authentication ` to get up + and running. + * Then configure your preferred authentication provider, such as :doc:`LDAP + `. + * Avoid the complexity of Kerberos for client authentication, if possible. + +#. **Enable authorization and access control** + + * Start with :doc:`file-based rules `. + * Then configure another access control method as required. + +Configure one step at a time. Always restart the Trino server after each +change, and verify the results before proceeding. + +.. _security-client: + +Securing client access to the cluster +------------------------------------- + +Trino :doc:`clients ` include the Trino :doc:`CLI `, +the :doc:`Web UI `, the :doc:`JDBC driver +`, `Python, Go, or other clients +`_, and any applications using these tools. + +All access to the Trino cluster is managed by the coordinator. Thus, securing +access to the cluster means securing access to the coordinator. + +There are three aspects to consider: + +* :ref:`cl-access-encrypt`: protecting the integrity of client to server + communication in transit. +* :ref:`cl-access-auth`: identifying users and user name management. +* :ref:`cl-access-control`: validating each user's access rights. + +.. _cl-access-encrypt: + +Encryption +^^^^^^^^^^ + +The Trino server uses the standard :doc:`HTTPS protocol and TLS encryption +`, formerly known as SSL. + +.. _cl-access-auth: + +Authentication +^^^^^^^^^^^^^^ + +Trino supports several authentication providers. When setting up a new cluster, +start with simple password file authentication before configuring another +provider. + +* :doc:`Password file authentication ` +* :doc:`LDAP authentication ` +* :doc:`OAuth 2.0 authentication ` +* :doc:`Salesforce authentication ` +* :doc:`Kerberos authentication ` + +.. _user-name-management: + +User name management +"""""""""""""""""""" + +Trino provides ways to map the user and group names from authentication +providers to Trino user names. + +* :doc:`User mapping ` applies to all authentication systems, + and allows for JSON files to specify rules to map complex user names from + other systems (``alice@example.com``) to simple user names (``alice``). +* :doc:`File group provider ` provides a way to assign a set + of user names to a group name to ease access control. + +.. _cl-access-control: + +Authorization and access control +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +Trino's :doc:`default method of access control ` +allows all operations for all authenticated users. + +To implement access control, use: + +* :doc:`File-based system access control `, where + you configure JSON files that specify fine-grained user access restrictions at + the catalog, schema, or table level. + +In addition, Trino :doc:`provides an API ` that +allows you to create a custom access control method, or to extend an existing +one. + +.. _security-inside-cluster: + +Securing inside the cluster +--------------------------- + +You can :doc:`secure the internal communication ` +between coordinator and workers inside the clusters. + +Secrets in properties files, such as passwords in catalog files, can be secured +with :doc:`secrets management `. + +.. _security-data-sources: + +Securing cluster access to data sources +--------------------------------------- + +Communication between the Trino cluster and data sources is configured for each +catalog. Each catalog uses a connector, which supports a variety of +security-related configurations. + +More information is available with the documentation for individual +:doc:`connectors `. + +:doc:`Secrets management ` can be used for the catalog properties files +content. + diff --git a/docs/src/main/sphinx/security/password-file.rst b/docs/src/main/sphinx/security/password-file.rst index 0f26b3e3e6a7..e5bc7f457e56 100644 --- a/docs/src/main/sphinx/security/password-file.rst +++ b/docs/src/main/sphinx/security/password-file.rst @@ -14,8 +14,15 @@ and clients to use TLS and authenticate with a username and password. Password authenticator configuration ------------------------------------ -Enable password file authentication by creating an -``etc/password-authenticator.properties`` file on the coordinator: +To enable password file authentication, set the :doc:`password authentication +type ` in ``etc/config.properties``: + +.. code-block:: properties + + http-server.authentication.type=PASSWORD + +In addition, create a ``etc/password-authenticator.properties`` file on the +coordinator with the ``file`` authenticator name: .. code-block:: text diff --git a/docs/src/main/sphinx/security/salesforce.rst b/docs/src/main/sphinx/security/salesforce.rst index 14c61421b5e0..5041c33335e6 100644 --- a/docs/src/main/sphinx/security/salesforce.rst +++ b/docs/src/main/sphinx/security/salesforce.rst @@ -19,8 +19,15 @@ basic credentials. This can also be used to secure the :ref:`Web UI ` in ``etc/config.properties``: + +.. code-block:: properties + + http-server.authentication.type=PASSWORD + +In addition, create a ``etc/password-authenticator.properties`` file on the +coordinator with the ``salesforce`` authenticator name: .. code-block:: properties diff --git a/docs/src/main/sphinx/security/server.rst b/docs/src/main/sphinx/security/server.rst index bf6d66595353..cb108f1665b0 100644 --- a/docs/src/main/sphinx/security/server.rst +++ b/docs/src/main/sphinx/security/server.rst @@ -46,12 +46,11 @@ In addition, the Trino coordinator needs a `keytab file .. include:: ktadd-note.fragment -Java keystore file for TLS -^^^^^^^^^^^^^^^^^^^^^^^^^^ +Configuration for TLS +^^^^^^^^^^^^^^^^^^^^^ -When using Kerberos authentication, access to the Trino coordinator should be -through HTTPS. You can do it by creating a :ref:`server_java_keystore` on the -coordinator. +When using Kerberos authentication, access to the Trino coordinator must be +through :doc:`HTTPS and TLS `. System access control plugin ---------------------------- @@ -68,10 +67,10 @@ Trino coordinator to use Kerberos authentication and HTTPS. After making the following environment changes, you can make the changes to the Trino configuration files. +* :doc:`/security/tls` * :ref:`server_kerberos_services` * :ref:`server_kerberos_configuration` * :ref:`server_kerberos_principals` -* :ref:`server_java_keystore` * :doc:`System Access Control Plugin ` config.properties diff --git a/docs/src/main/sphinx/security/tls.rst b/docs/src/main/sphinx/security/tls.rst index f5201ef7cf5c..76574b5efb8e 100644 --- a/docs/src/main/sphinx/security/tls.rst +++ b/docs/src/main/sphinx/security/tls.rst @@ -1,84 +1,322 @@ -============================== -Java keystores and truststores -============================== +============= +HTTPS and TLS +============= -.. _server_java_keystore: +Trino runs with no security by default. This allows you to connect to the server +using URLs that specify the HTTP protocol when using the Trino :doc:`CLI +`, the :doc:`Web UI `, or other +clients. -Java keystore file for TLS --------------------------- +This topic describes how to configure your Trino server to use :ref:`TLS +` to require clients to use the HTTPS connection protocol. All +authentication technologies supported by Trino require configuring TLS as the +foundational layer. -Access to the Trino coordinator must be through HTTPS when using Kerberos -and LDAP authentication. The Trino coordinator uses a :ref:`Java Keystore -` file for its TLS configuration. These keys are -generated using :command:`keytool` and stored in a Java Keystore file for the -Trino coordinator. +When configured to use TLS, a Trino server responds to client connections using +TLS 1.2 and TLS 1.3 certificates. The server rejects TLS 1.1, TLS 1.0, and all +SSL format certificates. -The alias in the :command:`keytool` command line should match the principal that the -Trino coordinator uses. +.. important:: -You'll be prompted for the first and last name. Use the Common Name that will -be used in the certificate. In this case, it should be the unqualified hostname -of the Trino coordinator. In the following example, you can see this in the prompt -that confirms the information is correct: + This page discusses only how to prepare the Trino server for secure client + connections from outside of the Trino cluster to its coordinator. + +See the :ref:`TLS Glossary ` to clarify unfamiliar terms. + +Approaches +---------- + +To configure Trino with TLS support, consider two alternative paths: + +* Use the :ref:`load balancer or proxy ` at your site + or cloud environment to terminate HTTPS/TLS. This approach is the simplest and + strongly preferred solution. + +* Secure the Trino :ref:`server directly `. This + requires you to obtain a valid certificate, and add it to the Trino + coordinator's configuration. + +.. _https-load-balancer: + +Use a load balancer to terminate HTTPS/TLS +------------------------------------------ + +Your site or cloud environment may already have a :ref:`load balancer ` +or proxy server configured and running with a valid, globally trusted TLS +certificate. In this case, you can work with your network administrators to set +up your Trino server behind the load balancer. The load balancer or proxy server +accepts TLS connections and forwards them to the Trino coordinator, which +typically runs with default HTTP configuration on the default port, 8080. + +When a load balancer accepts a TLS encrypted connection, it adds a +`forwarded +`_ +HTTP header to the request, such as: .. code-block:: text - keytool -genkeypair -alias trino -keyalg RSA -keystore keystore.jks - Enter keystore password: - Re-enter new password: - What is your first and last name? - [Unknown]: trino-coordinator.example.com - What is the name of your organizational unit? - [Unknown]: - What is the name of your organization? - [Unknown]: - What is the name of your City or Locality? - [Unknown]: - What is the name of your State or Province? - [Unknown]: - What is the two-letter country code for this unit? - [Unknown]: - Is CN=trino-coordinator.example.com, OU=Unknown, O=Unknown, L=Unknown, ST=Unknown, C=Unknown correct? - [no]: yes - - Enter key password for - (RETURN if same as keystore password): - -.. _cli_java_truststore: - -Java truststore file for TLS ----------------------------- - -Truststore files contain certificates of trusted TLS/SSL servers, or of -Certificate Authorities trusted to identify servers. For securing access -to the Trino coordinator through HTTPS, the clients can configure truststores. -For the Trino CLI to trust the Trino coordinator, the coordinator's certificate -must be imported to the CLI's truststore. - -You can either import the certificate to the default Java truststore, or to a -custom truststore. You should be careful, if you choose to use the default -one, since you may need to remove the certificates of CAs you do not deem trustworthy. - -You can use :command:`keytool` to import the certificate to the truststore. -In the example, we are going to import ``trino_certificate.cer`` to a custom -truststore ``trino_trust.jks``, and you get a prompt asking if the certificate -can be trusted or not. + X-Forwarded-Proto: https + +This tells the Trino coordinator to process the connection as if a TLS +connection has already been successfully negotiated for it. This is why you do +not need to configure ``http-server.https.enabled=true`` for a coordinator +behind a load balancer. + +However, to enable processing of such forwarded headers, the server's +:ref:`config properties file ` *must* include the following: .. code-block:: text - $ keytool -import -v -trustcacerts -alias trino_trust -file trino_certificate.cer -keystore trino_trust.jks -keypass + http-server.process-forwarded=true + +This completes any necessary configuration for using HTTPS with a load balancer. +Client tools can access Trino with the URL exposed by the load balancer. + +.. _https-secure-directly: + +Secure Trino directly +---------------------- + +Instead of the preferred mechanism of using an :ref:`external load balancer +`, you can secure the Trino coordinator itself. This +requires you to obtain and install a TLS :ref:`certificate `, and +configure Trino to use it for client connections. + +Add a TLS certificate +^^^^^^^^^^^^^^^^^^^^^ + +Obtain a TLS certificate file for use with your Trino server. Consider the +following types of certificates: + +* **Globally trusted certificates** — A certificate that is automatically + trusted by all browsers and clients. This is the easiest type to use because + you do not need to configure clients. Obtain a certificate of this type from: + + * A commercial certificate vendor + * Your cloud infrastructure provider + * A domain name registrar, such as Verisign or GoDaddy + * A free certificate generator, such as + `letsencrypt.org `_ or + `sslforfree.com `_ + +* **Corporate trusted certificates** — A certificate trusted by browsers and + clients in your organization. Typically, a site's IT department runs a local + :ref:`certificate authority ` and preconfigures clients and servers + to trust this CA. + +* **Generated self-signed certificates** — A certificate generated just for + Trino that is not automatically trusted by any client. Before using, make sure + you understand the :ref:`limitations of self-signed certificates + `. + +The most convenient option and strongly recommended option is a globally trusted +certificate. It may require a little more work up front, but it is worth it to +not have to configure every single client. -Troubleshooting ---------------- +Keys and certificates +^^^^^^^^^^^^^^^^^^^^^ -.. _troubleshooting_keystore: +Trino can read certificates and private keys encoded in PEM encoded PKCS #1, PEM +encoded PKCS #8, PKCS #12, and the legacy Java KeyStore (JKS) format. -Java keystore file verification -^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ +Make sure you obtain a certificate that is validated by a recognized +:ref:`certificate authority `. -Verify the password for a keystore file and view its contents using `keytool -`_. +Inspect received certificates +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +Before installing your certificate, inspect and validate the received key and +certificate files to make sure they reference the correct information to access +your Trino server. Much unnecessary debugging time is saved by taking the time +to validate your certificates before proceeding to configure the server. + +Inspect PEM-encoded files as described in :doc:`Inspect PEM files +`. + +Inspect PKCS # 12 and JKS keystores as described in :doc:`Inspect JKS files +`. + +Invalid certificates +^^^^^^^^^^^^^^^^^^^^^ + +If your certificate does not pass validation, or does not show the expected +information on inspection, contact the group or vendor who provided it for a +replacement. + +.. _cert-placement: + +Place the certificate file +^^^^^^^^^^^^^^^^^^^^^^^^^^ + +There are no location requirements for a certificate file as long as: + +* The file can be read by the Trino coordinator server process. +* The location is secure from copying or tampering by malicious actors. + +You can place your file in the Trino coordinator's ``etc`` directory, which +allows you to use a relative path reference in configuration files. However, +this location can require you to keep track of the certificate file, and move it +to a new ``etc`` directory when you upgrade your Trino version. + +.. _configure-https: + +Configure the coordinator +^^^^^^^^^^^^^^^^^^^^^^^^^ + +On the coordinator, add the following lines to the :ref:`config properties file +` to enable TLS/HTTPS support for the server. + +.. note:: + + Legacy ``keystore`` and ``truststore`` wording is used in property names, even + when directly using PEM-encoded certificates. + +.. code-block:: text + + http-server.https.enabled=true + http-server.https.port=8443 + http-server.https.keystore.path=etc/clustercoord.pem + +Possible alternatives for the third line include: .. code-block:: text - $ keytool -list -v -keystore /etc/trino/trino.jks + http-server.https.keystore.path=etc/clustercoord.jks + http-server.https.keystore.path=/usr/local/certs/clustercoord.p12 + +Relative paths are relative to the Trino server's root directory. In a +``tar.gz`` installation, the root directory is one level above ``etc``. + +JKS keystores always require a password, while PEM format certificates can +optionally require a password. For cases where you need a password, add the +following line to the configuration. + +.. code-block:: text + + http-server.https.keystore.key= + +It is possible for a key inside a keystore to have its own password, +independent of the keystore's password. In this case, specify the key's password +with the following property: + +.. code-block:: text + + http-server.https.keymanager.password= + +When your Trino coordinator has an authenticator enabled along with HTTPS +enabled, HTTP access is automatically disabled for all clients, including the +:doc:`Web UI `. Although not recommended, you can +re-enable it by setting: + +.. code-block:: text + + http-server.authentication.allow-insecure-over-http=true + +Test configuration +^^^^^^^^^^^^^^^^^^ + +To test your configuration settings, restart the server and try to connect to it +with the Trino :doc:`CLI ` or :doc:`Web UI +`, using a URL that begins with ``https://``. + +Now that TLS is configured, go back and :doc:`configure the authentication +` method for your server. + +.. _self_signed_limits: + +Limitations of self-signed certificates +--------------------------------------- + +It is possible to generate a self-signed certificate with the ``openssl``, +``keytool``, or on Linux, ``certtool`` commands. Self-signed certificates can be +useful during development of a cluster for internal use only. We recommend never +using a self-signed certificate for a production Trino server. + +Self-signed certificates are not trusted by anyone. They are typically created +by an administrator for expediency, because they do not require getting trust +signoff from anyone. + +To use a self-signed certificate while developing your cluster requires: + +* distributing to every client a local truststore that validates the certificate +* configuring every client to use this certificate + +However, even with this client configuration, modern browsers reject these +certificates, which makes self-signed servers difficult to work with. + +There is a difference between self-signed and unsigned certificates. Both types +are created with the same tools, but unsigned certificates are meant to be +forwarded to a CA with a Certificate Signing Request (CSR). The CA returns the +certificate signed by the CA and now globally trusted. + +.. _tlsglossary: + +TLS Glossary +------------ + +.. _glossCA: + +CA + Certificate Authority, a trusted organization that examines and validates + organizations and their proposed server URIs, and issues digital + certificates verified as valid for the requesting organization. + +.. _glossCert: + +Certificate + A public key `certificate + `_ issued by a CA, + sometimes abbreviated as *cert*, that verifies the ownership of a server's + keys. Certificate format is specified in the `X.509 + `_ standard. + +.. _glossJKS: + +JKS + Java KeyStore, the system of public key cryptography supported as one part + of the Java security APIs. The legacy JKS system recognizes keys and + certificates stored in *keystore* files, typically with the ``.jks`` + extension, and relies on a system-level list of CAs in *truststore* files + installed as part of the current Java installation. + +.. _glossKey: + +Key + A cryptographic key specified as a pair of public and private keys. + +.. _glossLB: + +Load Balancer (LB) + Software or a hardware device that sits on a network's outer edge or + firewall and accepts network connections on behalf of servers behind that + wall. Load balancers carefully manage network traffic, and can accept TLS + connections from incoming clients and pass those connections transparently + to servers behind the wall. + +.. _glossPEM: + +PEM + Privacy-Enhanced Mail, a syntax for private key information, and a content + type used to store and send cryptographic keys and certificates. PEM format + can contain both a key and its certificate, plus the chain of certificates + from authorities back to the root :ref:`CA `, or back to a CA + vendor's intermediate CA. + +.. _glossPKCS12: + +PKCS #12 + A binary archive used to store keys and certificates or certificate chains + that validate a key. `PKCS #12 `_ + files have ``.p12`` or ``.pfx`` extensions. + +SSL + Secure Sockets Layer, now superceded by TLS, but still recognized as the + term for what TLS does now. + +.. _glossTLS: + +TLS + `Transport Layer Security + `_ is the successor + to SSL. These security topics use the term TLS to refer to both TLS and SSL. + diff --git a/lib/trino-array/pom.xml b/lib/trino-array/pom.xml index 21d722280b35..3bf6c9b2a67a 100644 --- a/lib/trino-array/pom.xml +++ b/lib/trino-array/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/lib/trino-geospatial-toolkit/pom.xml b/lib/trino-geospatial-toolkit/pom.xml index 8f65e96c23d6..c002e8349fa8 100644 --- a/lib/trino-geospatial-toolkit/pom.xml +++ b/lib/trino-geospatial-toolkit/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/lib/trino-matching/pom.xml b/lib/trino-matching/pom.xml index d5c2b8c8752d..97d63f9ee261 100644 --- a/lib/trino-matching/pom.xml +++ b/lib/trino-matching/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/lib/trino-memory-context/pom.xml b/lib/trino-memory-context/pom.xml index 0dbd53e1d92f..119c7c51724a 100644 --- a/lib/trino-memory-context/pom.xml +++ b/lib/trino-memory-context/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/lib/trino-orc/pom.xml b/lib/trino-orc/pom.xml index 66509d0953b2..1467a966633e 100644 --- a/lib/trino-orc/pom.xml +++ b/lib/trino-orc/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/lib/trino-orc/src/main/java/io/trino/orc/OrcWriter.java b/lib/trino-orc/src/main/java/io/trino/orc/OrcWriter.java index 92cc86611172..714ebe7a8bac 100644 --- a/lib/trino-orc/src/main/java/io/trino/orc/OrcWriter.java +++ b/lib/trino-orc/src/main/java/io/trino/orc/OrcWriter.java @@ -88,13 +88,13 @@ public final class OrcWriter { private static final int INSTANCE_SIZE = ClassLayout.parseClass(OrcWriter.class).instanceSize(); - private static final String PRESTO_ORC_WRITER_VERSION_METADATA_KEY = "presto.writer.version"; - private static final String PRESTO_ORC_WRITER_VERSION; + private static final String TRINO_ORC_WRITER_VERSION_METADATA_KEY = "trino.writer.version"; + private static final String TRINO_ORC_WRITER_VERSION; private final OrcWriterStats stats; static { String version = OrcWriter.class.getPackage().getImplementationVersion(); - PRESTO_ORC_WRITER_VERSION = version == null ? "UNKNOWN" : version; + TRINO_ORC_WRITER_VERSION = version == null ? "UNKNOWN" : version; } private final OrcDataSink orcDataSink; @@ -135,7 +135,6 @@ public OrcWriter( ColumnMetadata orcTypes, CompressionKind compression, OrcWriterOptions options, - boolean writeLegacyVersion, Map userMetadata, boolean validate, OrcWriteValidationMode validationMode, @@ -161,8 +160,8 @@ public OrcWriter( this.maxCompressionBufferSize = toIntExact(options.getMaxCompressionBufferSize().toBytes()); this.userMetadata.putAll(requireNonNull(userMetadata, "userMetadata is null")); - this.userMetadata.put(PRESTO_ORC_WRITER_VERSION_METADATA_KEY, PRESTO_ORC_WRITER_VERSION); - this.metadataWriter = new CompressedMetadataWriter(new OrcMetadataWriter(writeLegacyVersion), compression, maxCompressionBufferSize); + this.userMetadata.put(TRINO_ORC_WRITER_VERSION_METADATA_KEY, TRINO_ORC_WRITER_VERSION); + this.metadataWriter = new CompressedMetadataWriter(new OrcMetadataWriter(options.getWriterIdentification()), compression, maxCompressionBufferSize); this.stats = requireNonNull(stats, "stats is null"); requireNonNull(columnNames, "columnNames is null"); @@ -530,7 +529,8 @@ private List bufferFileFooter() .collect(toImmutableList()), orcTypes, fileStats, - userMetadata); + userMetadata, + Optional.empty()); // writer id will be set by MetadataWriter closedStripes.clear(); closedStripesRetainedBytes = 0; diff --git a/lib/trino-orc/src/main/java/io/trino/orc/OrcWriterOptions.java b/lib/trino-orc/src/main/java/io/trino/orc/OrcWriterOptions.java index b1325936a483..8edbe42c03c3 100644 --- a/lib/trino-orc/src/main/java/io/trino/orc/OrcWriterOptions.java +++ b/lib/trino-orc/src/main/java/io/trino/orc/OrcWriterOptions.java @@ -27,16 +27,36 @@ public class OrcWriterOptions { + public enum WriterIdentification + { + /** + * Write ORC files with a writer identification and version number that is readable by Hive 2.0.0 to 2.2.0 + */ + LEGACY_HIVE_COMPATIBLE, + + /** + * Write ORC files with the legacy writer identification of PrestoSQL + */ + PRESTO, + + /** + * Write ORC files with Trino writer identification. + */ + TRINO, + } + @VisibleForTesting static final DataSize DEFAULT_MAX_STRING_STATISTICS_LIMIT = DataSize.ofBytes(64); @VisibleForTesting static final DataSize DEFAULT_MAX_COMPRESSION_BUFFER_SIZE = DataSize.of(256, KILOBYTE); - static final double DEFAULT_BLOOM_FILTER_FPP = 0.05; + private static final double DEFAULT_BLOOM_FILTER_FPP = 0.05; private static final DataSize DEFAULT_STRIPE_MIN_SIZE = DataSize.of(32, MEGABYTE); private static final DataSize DEFAULT_STRIPE_MAX_SIZE = DataSize.of(64, MEGABYTE); private static final int DEFAULT_STRIPE_MAX_ROW_COUNT = 10_000_000; private static final int DEFAULT_ROW_GROUP_MAX_ROW_COUNT = 10_000; private static final DataSize DEFAULT_DICTIONARY_MAX_MEMORY = DataSize.of(16, MEGABYTE); + + private final WriterIdentification writerIdentification; private final DataSize stripeMinSize; private final DataSize stripeMaxSize; private final int stripeMaxRowCount; @@ -50,6 +70,7 @@ public class OrcWriterOptions public OrcWriterOptions() { this( + WriterIdentification.TRINO, DEFAULT_STRIPE_MIN_SIZE, DEFAULT_STRIPE_MAX_SIZE, DEFAULT_STRIPE_MAX_ROW_COUNT, @@ -62,6 +83,7 @@ public OrcWriterOptions() } private OrcWriterOptions( + WriterIdentification writerIdentification, DataSize stripeMinSize, DataSize stripeMaxSize, int stripeMaxRowCount, @@ -82,6 +104,7 @@ private OrcWriterOptions( requireNonNull(bloomFilterColumns, "bloomFilterColumns is null"); checkArgument(bloomFilterFpp > 0.0 && bloomFilterFpp < 1.0, "bloomFilterFpp should be > 0.0 & < 1.0"); + this.writerIdentification = requireNonNull(writerIdentification, "writerIdentification is null"); this.stripeMinSize = stripeMinSize; this.stripeMaxSize = stripeMaxSize; this.stripeMaxRowCount = stripeMaxRowCount; @@ -93,94 +116,124 @@ private OrcWriterOptions( this.bloomFilterFpp = bloomFilterFpp; } - public DataSize getStripeMinSize() + public WriterIdentification getWriterIdentification() { - return stripeMinSize; + return writerIdentification; } - public DataSize getStripeMaxSize() + public OrcWriterOptions withWriterIdentification(WriterIdentification writerIdentification) { - return stripeMaxSize; + return builderFrom(this) + .setWriterIdentification(writerIdentification) + .build(); } - public int getStripeMaxRowCount() + public DataSize getStripeMinSize() { - return stripeMaxRowCount; + return stripeMinSize; } - public int getRowGroupMaxRowCount() + public OrcWriterOptions withStripeMinSize(DataSize stripeMinSize) { - return rowGroupMaxRowCount; + return builderFrom(this) + .setStripeMinSize(stripeMinSize) + .build(); } - public DataSize getDictionaryMaxMemory() + public DataSize getStripeMaxSize() { - return dictionaryMaxMemory; + return stripeMaxSize; } - public DataSize getMaxStringStatisticsLimit() + public OrcWriterOptions withStripeMaxSize(DataSize stripeMaxSize) { - return maxStringStatisticsLimit; + return builderFrom(this) + .setStripeMaxSize(stripeMaxSize) + .build(); } - public DataSize getMaxCompressionBufferSize() + public int getStripeMaxRowCount() { - return maxCompressionBufferSize; + return stripeMaxRowCount; } - public double getBloomFilterFpp() + public OrcWriterOptions withStripeMaxRowCount(int stripeMaxRowCount) { - return bloomFilterFpp; + return builderFrom(this) + .setStripeMaxRowCount(stripeMaxRowCount) + .build(); } - public boolean isBloomFilterColumn(String columnName) + public int getRowGroupMaxRowCount() { - return bloomFilterColumns.contains(columnName); + return rowGroupMaxRowCount; } - public OrcWriterOptions withStripeMinSize(DataSize stripeMinSize) + public OrcWriterOptions withRowGroupMaxRowCount(int rowGroupMaxRowCount) { - return new OrcWriterOptions(stripeMinSize, stripeMaxSize, stripeMaxRowCount, rowGroupMaxRowCount, dictionaryMaxMemory, maxStringStatisticsLimit, maxCompressionBufferSize, bloomFilterColumns, bloomFilterFpp); + return builderFrom(this) + .setRowGroupMaxRowCount(rowGroupMaxRowCount) + .build(); } - public OrcWriterOptions withStripeMaxSize(DataSize stripeMaxSize) + public DataSize getDictionaryMaxMemory() { - return new OrcWriterOptions(stripeMinSize, stripeMaxSize, stripeMaxRowCount, rowGroupMaxRowCount, dictionaryMaxMemory, maxStringStatisticsLimit, maxCompressionBufferSize, bloomFilterColumns, bloomFilterFpp); + return dictionaryMaxMemory; } - public OrcWriterOptions withStripeMaxRowCount(int stripeMaxRowCount) + public OrcWriterOptions withDictionaryMaxMemory(DataSize dictionaryMaxMemory) { - return new OrcWriterOptions(stripeMinSize, stripeMaxSize, stripeMaxRowCount, rowGroupMaxRowCount, dictionaryMaxMemory, maxStringStatisticsLimit, maxCompressionBufferSize, bloomFilterColumns, bloomFilterFpp); + return builderFrom(this) + .setDictionaryMaxMemory(dictionaryMaxMemory) + .build(); } - public OrcWriterOptions withRowGroupMaxRowCount(int rowGroupMaxRowCount) + public DataSize getMaxStringStatisticsLimit() { - return new OrcWriterOptions(stripeMinSize, stripeMaxSize, stripeMaxRowCount, rowGroupMaxRowCount, dictionaryMaxMemory, maxStringStatisticsLimit, maxCompressionBufferSize, bloomFilterColumns, bloomFilterFpp); + return maxStringStatisticsLimit; } - public OrcWriterOptions withDictionaryMaxMemory(DataSize dictionaryMaxMemory) + public OrcWriterOptions withMaxStringStatisticsLimit(DataSize maxStringStatisticsLimit) { - return new OrcWriterOptions(stripeMinSize, stripeMaxSize, stripeMaxRowCount, rowGroupMaxRowCount, dictionaryMaxMemory, maxStringStatisticsLimit, maxCompressionBufferSize, bloomFilterColumns, bloomFilterFpp); + return builderFrom(this) + .setMaxStringStatisticsLimit(maxStringStatisticsLimit) + .build(); } - public OrcWriterOptions withMaxStringStatisticsLimit(DataSize maxStringStatisticsLimit) + public DataSize getMaxCompressionBufferSize() { - return new OrcWriterOptions(stripeMinSize, stripeMaxSize, stripeMaxRowCount, rowGroupMaxRowCount, dictionaryMaxMemory, maxStringStatisticsLimit, maxCompressionBufferSize, bloomFilterColumns, bloomFilterFpp); + return maxCompressionBufferSize; } public OrcWriterOptions withMaxCompressionBufferSize(DataSize maxCompressionBufferSize) { - return new OrcWriterOptions(stripeMinSize, stripeMaxSize, stripeMaxRowCount, rowGroupMaxRowCount, dictionaryMaxMemory, maxStringStatisticsLimit, maxCompressionBufferSize, bloomFilterColumns, bloomFilterFpp); + return builderFrom(this) + .setMaxCompressionBufferSize(maxCompressionBufferSize) + .build(); + } + + public boolean isBloomFilterColumn(String columnName) + { + return bloomFilterColumns.contains(columnName); } public OrcWriterOptions withBloomFilterColumns(Set bloomFilterColumns) { - return new OrcWriterOptions(stripeMinSize, stripeMaxSize, stripeMaxRowCount, rowGroupMaxRowCount, dictionaryMaxMemory, maxStringStatisticsLimit, maxCompressionBufferSize, bloomFilterColumns, bloomFilterFpp); + return builderFrom(this) + .setBloomFilterColumns(bloomFilterColumns) + .build(); + } + + public double getBloomFilterFpp() + { + return bloomFilterFpp; } public OrcWriterOptions withBloomFilterFpp(double bloomFilterFpp) { - return new OrcWriterOptions(stripeMinSize, stripeMaxSize, stripeMaxRowCount, rowGroupMaxRowCount, dictionaryMaxMemory, maxStringStatisticsLimit, maxCompressionBufferSize, bloomFilterColumns, bloomFilterFpp); + return builderFrom(this) + .setBloomFilterFpp(bloomFilterFpp) + .build(); } @Override @@ -198,4 +251,119 @@ public String toString() .add("bloomFilterFpp", bloomFilterFpp) .toString(); } + + public static Builder builder() + { + return builderFrom(new OrcWriterOptions()); + } + + public static Builder builderFrom(OrcWriterOptions options) + { + return new Builder(options); + } + + public static final class Builder + { + private WriterIdentification writerIdentification; + private DataSize stripeMinSize; + private DataSize stripeMaxSize; + private int stripeMaxRowCount; + private int rowGroupMaxRowCount; + private DataSize dictionaryMaxMemory; + private DataSize maxStringStatisticsLimit; + private DataSize maxCompressionBufferSize; + private Set bloomFilterColumns; + private double bloomFilterFpp; + + private Builder(OrcWriterOptions options) + { + requireNonNull(options, "options is null"); + + this.writerIdentification = options.writerIdentification; + this.stripeMinSize = options.stripeMinSize; + this.stripeMaxSize = options.stripeMaxSize; + this.stripeMaxRowCount = options.stripeMaxRowCount; + this.rowGroupMaxRowCount = options.rowGroupMaxRowCount; + this.dictionaryMaxMemory = options.dictionaryMaxMemory; + this.maxStringStatisticsLimit = options.maxStringStatisticsLimit; + this.maxCompressionBufferSize = options.maxCompressionBufferSize; + this.bloomFilterColumns = ImmutableSet.copyOf(options.bloomFilterColumns); + this.bloomFilterFpp = options.bloomFilterFpp; + } + + public Builder setWriterIdentification(WriterIdentification writerIdentification) + { + this.writerIdentification = writerIdentification; + return this; + } + + public Builder setStripeMinSize(DataSize stripeMinSize) + { + this.stripeMinSize = stripeMinSize; + return this; + } + + public Builder setStripeMaxSize(DataSize stripeMaxSize) + { + this.stripeMaxSize = stripeMaxSize; + return this; + } + + public Builder setStripeMaxRowCount(int stripeMaxRowCount) + { + this.stripeMaxRowCount = stripeMaxRowCount; + return this; + } + + public Builder setRowGroupMaxRowCount(int rowGroupMaxRowCount) + { + this.rowGroupMaxRowCount = rowGroupMaxRowCount; + return this; + } + + public Builder setDictionaryMaxMemory(DataSize dictionaryMaxMemory) + { + this.dictionaryMaxMemory = dictionaryMaxMemory; + return this; + } + + public Builder setMaxStringStatisticsLimit(DataSize maxStringStatisticsLimit) + { + this.maxStringStatisticsLimit = maxStringStatisticsLimit; + return this; + } + + public Builder setMaxCompressionBufferSize(DataSize maxCompressionBufferSize) + { + this.maxCompressionBufferSize = maxCompressionBufferSize; + return this; + } + + public Builder setBloomFilterColumns(Set bloomFilterColumns) + { + this.bloomFilterColumns = bloomFilterColumns; + return this; + } + + public Builder setBloomFilterFpp(double bloomFilterFpp) + { + this.bloomFilterFpp = bloomFilterFpp; + return this; + } + + public OrcWriterOptions build() + { + return new OrcWriterOptions( + writerIdentification, + stripeMinSize, + stripeMaxSize, + stripeMaxRowCount, + rowGroupMaxRowCount, + dictionaryMaxMemory, + maxStringStatisticsLimit, + maxCompressionBufferSize, + bloomFilterColumns, + bloomFilterFpp); + } + } } diff --git a/lib/trino-orc/src/main/java/io/trino/orc/metadata/Footer.java b/lib/trino-orc/src/main/java/io/trino/orc/metadata/Footer.java index 52421cf542ca..7d8453f0b6a3 100644 --- a/lib/trino-orc/src/main/java/io/trino/orc/metadata/Footer.java +++ b/lib/trino-orc/src/main/java/io/trino/orc/metadata/Footer.java @@ -37,6 +37,7 @@ public class Footer private final ColumnMetadata types; private final Optional> fileStats; private final Map userMetadata; + private final Optional writerId; public Footer( long numberOfRows, @@ -44,7 +45,8 @@ public Footer( List stripes, ColumnMetadata types, Optional> fileStats, - Map userMetadata) + Map userMetadata, + Optional writerId) { this.numberOfRows = numberOfRows; rowsInRowGroup.ifPresent(value -> checkArgument(value > 0, "rowsInRowGroup must be at least 1")); @@ -54,6 +56,7 @@ public Footer( this.fileStats = requireNonNull(fileStats, "fileStats is null"); requireNonNull(userMetadata, "userMetadata is null"); this.userMetadata = ImmutableMap.copyOf(transformValues(userMetadata, Slices::copyOf)); + this.writerId = requireNonNull(writerId, "writerId is null"); } public long getNumberOfRows() @@ -86,6 +89,11 @@ public Map getUserMetadata() return ImmutableMap.copyOf(transformValues(userMetadata, Slices::copyOf)); } + public Optional getWriterId() + { + return writerId; + } + @Override public String toString() { @@ -96,6 +104,7 @@ public String toString() .add("types", types) .add("columnStatistics", fileStats) .add("userMetadata", userMetadata.keySet()) + .add("writerId", writerId) .toString(); } } diff --git a/lib/trino-orc/src/main/java/io/trino/orc/metadata/OrcMetadataReader.java b/lib/trino-orc/src/main/java/io/trino/orc/metadata/OrcMetadataReader.java index 176e91f76a86..9b410a3be27d 100644 --- a/lib/trino-orc/src/main/java/io/trino/orc/metadata/OrcMetadataReader.java +++ b/lib/trino-orc/src/main/java/io/trino/orc/metadata/OrcMetadataReader.java @@ -141,7 +141,8 @@ public Footer readFooter(HiveWriterVersion hiveWriterVersion, InputStream inputS toStripeInformation(footer.getStripesList()), toType(footer.getTypesList()), toColumnStatistics(hiveWriterVersion, footer.getStatisticsList(), false), - toUserMetadata(footer.getMetadataList())); + toUserMetadata(footer.getMetadataList()), + Optional.of(footer.getWriter())); } private static List toStripeInformation(List types) diff --git a/lib/trino-orc/src/main/java/io/trino/orc/metadata/OrcMetadataWriter.java b/lib/trino-orc/src/main/java/io/trino/orc/metadata/OrcMetadataWriter.java index 1d77a83eab87..09d1600e4360 100644 --- a/lib/trino-orc/src/main/java/io/trino/orc/metadata/OrcMetadataWriter.java +++ b/lib/trino-orc/src/main/java/io/trino/orc/metadata/OrcMetadataWriter.java @@ -18,6 +18,7 @@ import com.google.common.primitives.Longs; import io.airlift.slice.Slice; import io.airlift.slice.SliceOutput; +import io.trino.orc.OrcWriterOptions.WriterIdentification; import io.trino.orc.metadata.ColumnEncoding.ColumnEncodingKind; import io.trino.orc.metadata.OrcType.OrcTypeKind; import io.trino.orc.metadata.Stream.StreamKind; @@ -42,24 +43,32 @@ import static com.google.common.collect.ImmutableList.toImmutableList; import static io.trino.orc.metadata.PostScript.MAGIC; import static java.lang.Math.toIntExact; +import static java.util.Objects.requireNonNull; import static java.util.stream.Collectors.toList; public class OrcMetadataWriter implements MetadataWriter { // see https://github.com/trinodb/orc-protobuf/blob/master/src/main/protobuf/orc_proto.proto - private static final int PRESTO_WRITER_ID = 2; + public static final int TRINO_WRITER_ID = 4; + // in order to change this value, the master Apache ORC proto file must be updated + private static final int TRINO_WRITER_VERSION = 6; + + // see https://github.com/trinodb/orc-protobuf/blob/master/src/main/protobuf/orc_proto.proto + public static final int PRESTO_WRITER_ID = 2; // in order to change this value, the master Apache ORC proto file must be updated private static final int PRESTO_WRITER_VERSION = 6; + // maximum version readable by Hive 2.x before the ORC-125 fix private static final int HIVE_LEGACY_WRITER_VERSION = 4; + private static final List ORC_METADATA_VERSION = ImmutableList.of(0, 12); - private final boolean useLegacyVersion; + private final WriterIdentification writerIdentification; - public OrcMetadataWriter(boolean useLegacyVersion) + public OrcMetadataWriter(WriterIdentification writerIdentification) { - this.useLegacyVersion = useLegacyVersion; + this.writerIdentification = requireNonNull(writerIdentification, "writerIdentification is null"); } @Override @@ -78,13 +87,26 @@ public int writePostscript(SliceOutput output, int footerLength, int metadataLen .setMetadataLength(metadataLength) .setCompression(toCompression(compression)) .setCompressionBlockSize(compressionBlockSize) - .setWriterVersion(useLegacyVersion ? HIVE_LEGACY_WRITER_VERSION : PRESTO_WRITER_VERSION) + .setWriterVersion(getOrcWriterVersion()) .setMagic(MAGIC.toStringUtf8()) .build(); return writeProtobufObject(output, postScriptProtobuf); } + private int getOrcWriterVersion() + { + switch (writerIdentification) { + case LEGACY_HIVE_COMPATIBLE: + return HIVE_LEGACY_WRITER_VERSION; + case PRESTO: + return PRESTO_WRITER_VERSION; + case TRINO: + return TRINO_WRITER_VERSION; + } + throw new IllegalStateException("Unexpected value: " + writerIdentification); + } + @Override public int writeMetadata(SliceOutput output, Metadata metadata) throws IOException @@ -128,13 +150,28 @@ public int writeFooter(SliceOutput output, Footer footer) .map(OrcMetadataWriter::toUserMetadata) .collect(toList())); - if (!useLegacyVersion) { - builder.setWriter(PRESTO_WRITER_ID); - } + setWriter(builder); return writeProtobufObject(output, builder.build()); } + private void setWriter(OrcProto.Footer.Builder builder) + { + switch (writerIdentification) { + case LEGACY_HIVE_COMPATIBLE: + return; + + case PRESTO: + builder.setWriter(PRESTO_WRITER_ID); + return; + + case TRINO: + builder.setWriter(TRINO_WRITER_ID); + return; + } + throw new IllegalStateException("Unexpected value: " + writerIdentification); + } + private static OrcProto.StripeInformation toStripeInformation(StripeInformation stripe) { return OrcProto.StripeInformation.newBuilder() diff --git a/lib/trino-orc/src/test/java/io/trino/orc/OrcTester.java b/lib/trino-orc/src/test/java/io/trino/orc/OrcTester.java index d581ee503afb..7d9c65d246ee 100644 --- a/lib/trino-orc/src/test/java/io/trino/orc/OrcTester.java +++ b/lib/trino-orc/src/test/java/io/trino/orc/OrcTester.java @@ -618,7 +618,6 @@ public static void writeOrcPages(File outputFile, CompressionKind compression, L OrcType.createRootOrcType(columnNames, types), compression, new OrcWriterOptions(), - false, ImmutableMap.of(), true, BOTH, @@ -645,7 +644,6 @@ public static void writeOrcColumnTrino(File outputFile, CompressionKind compress OrcType.createRootOrcType(columnNames, types), compression, new OrcWriterOptions(), - false, ImmutableMap.of(), true, BOTH, diff --git a/lib/trino-orc/src/test/java/io/trino/orc/TestOrcBloomFilters.java b/lib/trino-orc/src/test/java/io/trino/orc/TestOrcBloomFilters.java index 222c38acc801..b6dc072e4b88 100644 --- a/lib/trino-orc/src/test/java/io/trino/orc/TestOrcBloomFilters.java +++ b/lib/trino-orc/src/test/java/io/trino/orc/TestOrcBloomFilters.java @@ -17,6 +17,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.primitives.Longs; import io.airlift.slice.Slice; +import io.trino.orc.OrcWriterOptions.WriterIdentification; import io.trino.orc.metadata.ColumnMetadata; import io.trino.orc.metadata.CompressedMetadataWriter; import io.trino.orc.metadata.CompressionKind; @@ -123,7 +124,7 @@ public void testOrcHiveBloomFilterSerde() assertTrue(bloomFilterWrite.test(TEST_STRING)); assertTrue(bloomFilterWrite.testSlice(wrappedBuffer(TEST_STRING))); - Slice bloomFilterBytes = new CompressedMetadataWriter(new OrcMetadataWriter(true), CompressionKind.NONE, 1024) + Slice bloomFilterBytes = new CompressedMetadataWriter(new OrcMetadataWriter(WriterIdentification.TRINO), CompressionKind.NONE, 1024) .writeBloomFilters(ImmutableList.of(bloomFilterWrite)); // Read through method diff --git a/lib/trino-orc/src/test/java/io/trino/orc/TestOrcWriter.java b/lib/trino-orc/src/test/java/io/trino/orc/TestOrcWriter.java index 0c3c0c16cf39..044b167e9faa 100644 --- a/lib/trino-orc/src/test/java/io/trino/orc/TestOrcWriter.java +++ b/lib/trino-orc/src/test/java/io/trino/orc/TestOrcWriter.java @@ -79,7 +79,6 @@ public void testWriteOutputStreamsInOrder() .withRowGroupMaxRowCount(ORC_ROW_GROUP_SIZE) .withDictionaryMaxMemory(DataSize.of(32, MEGABYTE)) .withBloomFilterColumns(ImmutableSet.copyOf(columnNames)), - false, ImmutableMap.of(), true, validationMode, diff --git a/lib/trino-orc/src/test/java/io/trino/orc/TestStructColumnReader.java b/lib/trino-orc/src/test/java/io/trino/orc/TestStructColumnReader.java index bb5f135c1416..fa7552c65e0b 100644 --- a/lib/trino-orc/src/test/java/io/trino/orc/TestStructColumnReader.java +++ b/lib/trino-orc/src/test/java/io/trino/orc/TestStructColumnReader.java @@ -228,7 +228,6 @@ private void write(TempFile tempFile, Type writerType, List data) .withStripeMaxRowCount(ORC_STRIPE_SIZE) .withRowGroupMaxRowCount(ORC_ROW_GROUP_SIZE) .withDictionaryMaxMemory(DataSize.of(32, MEGABYTE)), - false, ImmutableMap.of(), true, BOTH, diff --git a/lib/trino-parquet/pom.xml b/lib/trino-parquet/pom.xml index 28098fd4b104..e3fe9f3fb49d 100644 --- a/lib/trino-parquet/pom.xml +++ b/lib/trino-parquet/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/lib/trino-parquet/src/main/java/io/trino/parquet/reader/ParquetReader.java b/lib/trino-parquet/src/main/java/io/trino/parquet/reader/ParquetReader.java index 5e7b4fe5ad7b..51f403df0277 100644 --- a/lib/trino-parquet/src/main/java/io/trino/parquet/reader/ParquetReader.java +++ b/lib/trino-parquet/src/main/java/io/trino/parquet/reader/ParquetReader.java @@ -71,6 +71,7 @@ public class ParquetReader private final Optional fileCreatedBy; private final List blocks; + private final Optional> firstRowsOfBlocks; private final List columns; private final ParquetDataSource dataSource; private final DateTimeZone timeZone; @@ -79,6 +80,13 @@ public class ParquetReader private int currentRowGroup = -1; private BlockMetaData currentBlockMetadata; private long currentGroupRowCount; + /** + * Index in the Parquet file of the first row of the current group + */ + private Optional firstRowIndexInGroup = Optional.empty(); + /** + * Index in the current group of the next row + */ private long nextRowInGroup; private int batchSize; private int nextBatchSize = INITIAL_BATCH_SIZE; @@ -95,6 +103,7 @@ public ParquetReader( Optional fileCreatedBy, MessageColumnIO messageColumnIO, List blocks, + Optional> firstRowsOfBlocks, ParquetDataSource dataSource, DateTimeZone timeZone, AggregatedMemoryContext systemMemoryContext, @@ -104,6 +113,7 @@ public ParquetReader( this.fileCreatedBy = requireNonNull(fileCreatedBy, "fileCreatedBy is null"); this.columns = requireNonNull(messageColumnIO, "messageColumnIO is null").getLeaves(); this.blocks = requireNonNull(blocks, "blocks is null"); + this.firstRowsOfBlocks = requireNonNull(firstRowsOfBlocks, "firstRowsOfBlocks is null"); this.dataSource = requireNonNull(dataSource, "dataSource is null"); this.timeZone = requireNonNull(timeZone, "timeZone is null"); this.systemMemoryContext = requireNonNull(systemMemoryContext, "systemMemoryContext is null"); @@ -112,6 +122,10 @@ public ParquetReader( this.columnReaders = new PrimitiveColumnReader[columns.size()]; this.maxBytesPerCell = new long[columns.size()]; + firstRowsOfBlocks.ifPresent(firstRows -> { + checkArgument(blocks.size() == firstRows.size(), "elements of firstRowsOfBlocks must correspond to blocks"); + }); + Map ranges = new HashMap<>(); for (int rowGroup = 0; rowGroup < blocks.size(); rowGroup++) { BlockMetaData metadata = blocks.get(rowGroup); @@ -135,6 +149,15 @@ public void close() dataSource.close(); } + /** + * Get the global row index of the first row in the last batch. + */ + public long lastBatchStartRow() + { + long baseIndex = firstRowIndexInGroup.orElseThrow(() -> new IllegalStateException("row index unavailable")); + return baseIndex + nextRowInGroup - batchSize; + } + public int nextBatch() { if (nextRowInGroup >= currentGroupRowCount && !advanceToNextRowGroup()) { @@ -162,7 +185,7 @@ private boolean advanceToNextRowGroup() return false; } currentBlockMetadata = blocks.get(currentRowGroup); - + firstRowIndexInGroup = firstRowsOfBlocks.map(firstRows -> firstRows.get(currentRowGroup)); nextRowInGroup = 0L; currentGroupRowCount = currentBlockMetadata.getRowCount(); initializeColumnReaders(); diff --git a/lib/trino-plugin-toolkit/pom.xml b/lib/trino-plugin-toolkit/pom.xml index 63cacfaccc2c..b6bf9533aa2f 100644 --- a/lib/trino-plugin-toolkit/pom.xml +++ b/lib/trino-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/classloader/ClassLoaderSafeConnectorMetadata.java b/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/classloader/ClassLoaderSafeConnectorMetadata.java index 6c3e5a6257aa..1bb31955d5dc 100644 --- a/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/classloader/ClassLoaderSafeConnectorMetadata.java +++ b/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/classloader/ClassLoaderSafeConnectorMetadata.java @@ -35,6 +35,7 @@ import io.trino.spi.connector.ConnectorTableLayoutResult; import io.trino.spi.connector.ConnectorTableMetadata; import io.trino.spi.connector.ConnectorTableProperties; +import io.trino.spi.connector.ConnectorTableSchema; import io.trino.spi.connector.ConnectorViewDefinition; import io.trino.spi.connector.Constraint; import io.trino.spi.connector.ConstraintApplicationResult; @@ -218,6 +219,14 @@ public Optional getSystemTable(ConnectorSession session, SchemaTabl } } + @Override + public ConnectorTableSchema getTableSchema(ConnectorSession session, ConnectorTableHandle table) + { + try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { + return delegate.getTableSchema(session, table); + } + } + @Override public ConnectorTableMetadata getTableMetadata(ConnectorSession session, ConnectorTableHandle table) { diff --git a/lib/trino-rcfile/pom.xml b/lib/trino-rcfile/pom.xml index e974ef200c1c..5a5331ac4121 100644 --- a/lib/trino-rcfile/pom.xml +++ b/lib/trino-rcfile/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/lib/trino-record-decoder/pom.xml b/lib/trino-record-decoder/pom.xml index 522ff3b3bf39..8d0926b0d4fa 100644 --- a/lib/trino-record-decoder/pom.xml +++ b/lib/trino-record-decoder/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-accumulo-iterators/pom.xml b/plugin/trino-accumulo-iterators/pom.xml index 9707a3d621e6..56f725e86739 100644 --- a/plugin/trino-accumulo-iterators/pom.xml +++ b/plugin/trino-accumulo-iterators/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-accumulo/pom.xml b/plugin/trino-accumulo/pom.xml index 4c943110c70b..a7261dc6ce44 100644 --- a/plugin/trino-accumulo/pom.xml +++ b/plugin/trino-accumulo/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-atop/pom.xml b/plugin/trino-atop/pom.xml index c326df81e0cf..ab85e6475d99 100644 --- a/plugin/trino-atop/pom.xml +++ b/plugin/trino-atop/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-base-jdbc/pom.xml b/plugin/trino-base-jdbc/pom.xml index 2ce04f21a32b..7a3d593613c6 100644 --- a/plugin/trino-base-jdbc/pom.xml +++ b/plugin/trino-base-jdbc/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/CachingJdbcClient.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/CachingJdbcClient.java index e4adbc4d2dba..3bbf15c84901 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/CachingJdbcClient.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/CachingJdbcClient.java @@ -21,6 +21,7 @@ import com.google.common.util.concurrent.UncheckedExecutionException; import io.airlift.units.Duration; import io.trino.plugin.base.session.SessionPropertiesProvider; +import io.trino.plugin.jdbc.JdbcIdentityCacheMapping.JdbcIdentityCacheKey; import io.trino.spi.TrinoException; import io.trino.spi.connector.AggregateFunction; import io.trino.spi.connector.ColumnHandle; @@ -50,7 +51,6 @@ import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; import java.util.function.Predicate; import static com.google.common.base.MoreObjects.firstNonNull; @@ -60,40 +60,51 @@ import static com.google.common.collect.ImmutableMap.toImmutableMap; import static com.google.common.collect.ImmutableSet.toImmutableSet; import static java.util.Objects.requireNonNull; +import static java.util.concurrent.TimeUnit.MILLISECONDS; public class CachingJdbcClient implements JdbcClient { private static final Object NULL_MARKER = new Object(); - private static final Duration CACHING_DISABLED = Duration.valueOf("0ms"); + private static final Duration CACHING_DISABLED = new Duration(0, MILLISECONDS); private final JdbcClient delegate; + private final List> sessionProperties; private final boolean cacheMissing; + private final JdbcIdentityCacheMapping identityMapping; - private final Cache> schemaNamesCache; + private final Cache> schemaNamesCache; private final Cache> tableNamesCache; private final Cache> tableHandleCache; private final Cache> columnsCache; - private final List> sessionProperties; private final Cache statisticsCache; @Inject - public CachingJdbcClient(@StatsCollecting JdbcClient delegate, Set sessionPropertiesProviders, BaseJdbcConfig config) + public CachingJdbcClient( + @StatsCollecting JdbcClient delegate, + Set sessionPropertiesProviders, + JdbcIdentityCacheMapping identityMapping, + BaseJdbcConfig config) { - this(delegate, sessionPropertiesProviders, config.getMetadataCacheTtl(), config.isCacheMissing()); + this(delegate, sessionPropertiesProviders, identityMapping, config.getMetadataCacheTtl(), config.isCacheMissing()); } - public CachingJdbcClient(JdbcClient delegate, Set sessionPropertiesProviders, Duration metadataCachingTtl, boolean cacheMissing) + public CachingJdbcClient( + JdbcClient delegate, + Set sessionPropertiesProviders, + JdbcIdentityCacheMapping identityMapping, + Duration metadataCachingTtl, + boolean cacheMissing) { this.delegate = requireNonNull(delegate, "delegate is null"); this.sessionProperties = requireNonNull(sessionPropertiesProviders, "sessionPropertiesProviders is null").stream() .flatMap(provider -> provider.getSessionProperties().stream()) .collect(toImmutableList()); - this.cacheMissing = cacheMissing; + this.identityMapping = requireNonNull(identityMapping, "identityMapping is null"); CacheBuilder cacheBuilder = CacheBuilder.newBuilder() - .expireAfterWrite(metadataCachingTtl.toMillis(), TimeUnit.MILLISECONDS) + .expireAfterWrite(metadataCachingTtl.toMillis(), MILLISECONDS) .recordStats(); if (metadataCachingTtl.equals(CACHING_DISABLED)) { @@ -118,14 +129,14 @@ public boolean schemaExists(ConnectorSession session, String schema) @Override public Set getSchemaNames(ConnectorSession session) { - JdbcIdentity key = JdbcIdentity.from(session); + JdbcIdentityCacheKey key = getIdentityKey(session); return get(schemaNamesCache, key, () -> delegate.getSchemaNames(session)); } @Override public List getTableNames(ConnectorSession session, Optional schema) { - TableNamesCacheKey key = new TableNamesCacheKey(JdbcIdentity.from(session), schema); + TableNamesCacheKey key = new TableNamesCacheKey(getIdentityKey(session), schema); return get(tableNamesCache, key, () -> delegate.getTableNames(session, schema)); } @@ -135,7 +146,7 @@ public List getColumns(ConnectorSession session, JdbcTableHand if (tableHandle.getColumns().isPresent()) { return tableHandle.getColumns().get(); } - ColumnsCacheKey key = new ColumnsCacheKey(JdbcIdentity.from(session), getSessionProperties(session), tableHandle.getRequiredNamedRelation().getSchemaTableName()); + ColumnsCacheKey key = new ColumnsCacheKey(getIdentityKey(session), getSessionProperties(session), tableHandle.getRequiredNamedRelation().getSchemaTableName()); return get(columnsCache, key, () -> delegate.getColumns(session, tableHandle)); } @@ -248,7 +259,7 @@ public boolean isLimitGuaranteed(ConnectorSession session) @Override public Optional getTableHandle(ConnectorSession session, SchemaTableName schemaTableName) { - TableHandleCacheKey key = new TableHandleCacheKey(JdbcIdentity.from(session), schemaTableName); + TableHandleCacheKey key = new TableHandleCacheKey(getIdentityKey(session), schemaTableName); Optional cachedTableHandle = tableHandleCache.getIfPresent(key); //noinspection OptionalAssignedToNull if (cachedTableHandle != null) { @@ -424,6 +435,11 @@ public Optional getTableScanRedirection(Conn return delegate.getTableScanRedirection(session, tableHandle); } + private JdbcIdentityCacheKey getIdentityKey(ConnectorSession session) + { + return identityMapping.getRemoteUserCacheKey(JdbcIdentity.from(session)); + } + private Map getSessionProperties(ConnectorSession session) { return sessionProperties.stream() @@ -454,6 +470,12 @@ private void invalidateColumnsCache(SchemaTableName table) invalidateCache(columnsCache, key -> key.table.equals(table)); } + @VisibleForTesting + CacheStats getTableNamesCacheStats() + { + return tableNamesCache.stats(); + } + @VisibleForTesting CacheStats getColumnsCacheStats() { @@ -477,18 +499,18 @@ private static void invalidateCache(Cache cache, Predicate filte private static final class ColumnsCacheKey { - private final JdbcIdentity identity; + private final JdbcIdentityCacheKey identity; private final SchemaTableName table; private final Map sessionProperties; - private ColumnsCacheKey(JdbcIdentity identity, Map sessionProperties, SchemaTableName table) + private ColumnsCacheKey(JdbcIdentityCacheKey identity, Map sessionProperties, SchemaTableName table) { this.identity = requireNonNull(identity, "identity is null"); this.sessionProperties = ImmutableMap.copyOf(requireNonNull(sessionProperties, "sessionProperties is null")); this.table = requireNonNull(table, "table is null"); } - public JdbcIdentity getIdentity() + public JdbcIdentityCacheKey getIdentity() { return identity; } @@ -527,10 +549,10 @@ public String toString() private static final class TableHandleCacheKey { - private final JdbcIdentity identity; + private final JdbcIdentityCacheKey identity; private final SchemaTableName tableName; - private TableHandleCacheKey(JdbcIdentity identity, SchemaTableName tableName) + private TableHandleCacheKey(JdbcIdentityCacheKey identity, SchemaTableName tableName) { this.identity = requireNonNull(identity, "identity is null"); this.tableName = requireNonNull(tableName, "tableName is null"); @@ -559,10 +581,10 @@ public int hashCode() private static final class TableNamesCacheKey { - private final JdbcIdentity identity; + private final JdbcIdentityCacheKey identity; private final Optional schemaName; - private TableNamesCacheKey(JdbcIdentity identity, Optional schemaName) + private TableNamesCacheKey(JdbcIdentityCacheKey identity, Optional schemaName) { this.identity = requireNonNull(identity, "identity is null"); this.schemaName = requireNonNull(schemaName, "schemaName is null"); diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/ExtraCredentialsBasedJdbcIdentityCacheMapping.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/ExtraCredentialsBasedJdbcIdentityCacheMapping.java new file mode 100644 index 000000000000..e87fdf987a46 --- /dev/null +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/ExtraCredentialsBasedJdbcIdentityCacheMapping.java @@ -0,0 +1,102 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.trino.plugin.jdbc; + +import io.trino.plugin.jdbc.credential.ExtraCredentialConfig; + +import javax.inject.Inject; + +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; +import java.util.Arrays; +import java.util.Map; +import java.util.Optional; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static java.util.Objects.requireNonNull; + +public final class ExtraCredentialsBasedJdbcIdentityCacheMapping + implements JdbcIdentityCacheMapping +{ + private final MessageDigest sha256; + private final Optional userCredentialName; + private final Optional passwordCredentialName; + + @Inject + public ExtraCredentialsBasedJdbcIdentityCacheMapping(ExtraCredentialConfig config) + { + try { + sha256 = MessageDigest.getInstance("SHA-256"); + } + catch (NoSuchAlgorithmException e) { + throw new RuntimeException(e); + } + requireNonNull(config, "config is null"); + userCredentialName = config.getUserCredentialName(); + passwordCredentialName = config.getPasswordCredentialName(); + } + + @Override + public JdbcIdentityCacheKey getRemoteUserCacheKey(JdbcIdentity identity) + { + Map extraCredentials = identity.getExtraCredentials(); + return new ExtraCredentialsBasedJdbcIdentityCacheKey( + userCredentialName.map(extraCredentials::get) + .map(this::hash), + passwordCredentialName.map(extraCredentials::get) + .map(this::hash)); + } + + private byte[] hash(String value) + { + return sha256.digest(value.getBytes(UTF_8)); + } + + private static final class ExtraCredentialsBasedJdbcIdentityCacheKey + extends JdbcIdentityCacheKey + { + private static final byte[] EMPTY_BYTES = new byte[0]; + private final byte[] userHash; + private final byte[] passwordHash; + + public ExtraCredentialsBasedJdbcIdentityCacheKey(Optional userHash, Optional passwordHash) + { + this.userHash = requireNonNull(userHash, "userHash is null") + .orElse(EMPTY_BYTES); + this.passwordHash = requireNonNull(passwordHash, "passwordHash is null") + .orElse(EMPTY_BYTES); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ExtraCredentialsBasedJdbcIdentityCacheKey that = (ExtraCredentialsBasedJdbcIdentityCacheKey) o; + return Arrays.equals(userHash, that.userHash) && Arrays.equals(passwordHash, that.passwordHash); + } + + @Override + public int hashCode() + { + int result = Arrays.hashCode(userHash); + result = 31 * result + Arrays.hashCode(passwordHash); + return result; + } + } +} diff --git a/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/KafkaConsumerModule.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/ExtraCredentialsBasedJdbcIdentityCacheMappingModule.java similarity index 77% rename from plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/KafkaConsumerModule.java rename to plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/ExtraCredentialsBasedJdbcIdentityCacheMappingModule.java index 0fc43c41215e..065cf8ef1b02 100644 --- a/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/KafkaConsumerModule.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/ExtraCredentialsBasedJdbcIdentityCacheMappingModule.java @@ -11,18 +11,18 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.kafka; +package io.trino.plugin.jdbc; import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.Scopes; -public class KafkaConsumerModule +public class ExtraCredentialsBasedJdbcIdentityCacheMappingModule implements Module { @Override public void configure(Binder binder) { - binder.bind(KafkaConsumerFactory.class).to(PlainTextKafkaConsumerFactory.class).in(Scopes.SINGLETON); + binder.bind(JdbcIdentityCacheMapping.class).to(ExtraCredentialsBasedJdbcIdentityCacheMapping.class).in(Scopes.SINGLETON); } } diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/ForRecordCursor.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/ForRecordCursor.java new file mode 100644 index 000000000000..39f6292f917d --- /dev/null +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/ForRecordCursor.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 io.trino.plugin.jdbc; + +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 ForRecordCursor +{ +} diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcColumnHandle.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcColumnHandle.java index 22b7231c1590..8f8cac58b577 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcColumnHandle.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcColumnHandle.java @@ -18,6 +18,7 @@ import com.google.common.base.Joiner; import io.trino.spi.connector.ColumnHandle; import io.trino.spi.connector.ColumnMetadata; +import io.trino.spi.connector.ColumnSchema; import io.trino.spi.type.Type; import java.util.Objects; @@ -108,6 +109,14 @@ public ColumnMetadata getColumnMetadata() .build(); } + public ColumnSchema getColumnSchema() + { + return ColumnSchema.builder() + .setName(columnName) + .setType(columnType) + .build(); + } + @Override public boolean equals(Object obj) { diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcIdentityCacheMapping.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcIdentityCacheMapping.java new file mode 100644 index 000000000000..b505d8b40684 --- /dev/null +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcIdentityCacheMapping.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 io.trino.plugin.jdbc; + +public interface JdbcIdentityCacheMapping +{ + JdbcIdentityCacheKey getRemoteUserCacheKey(JdbcIdentity identity); + + /** + * This will be used as cache key for metadata. If {@link JdbcIdentity} content can influence the + * metadata then we should have {@link JdbcIdentityCacheKey} instance so + * we could cache proper metadata for given {@link JdbcIdentity}. + */ + abstract class JdbcIdentityCacheKey + { + @Override + public abstract int hashCode(); + + @Override + public abstract boolean equals(Object obj); + } +} diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcMetadata.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcMetadata.java index f45ecf9d3a38..0dfafec7e428 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcMetadata.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcMetadata.java @@ -33,6 +33,7 @@ import io.trino.spi.connector.ConnectorTableHandle; import io.trino.spi.connector.ConnectorTableMetadata; import io.trino.spi.connector.ConnectorTableProperties; +import io.trino.spi.connector.ConnectorTableSchema; import io.trino.spi.connector.Constraint; import io.trino.spi.connector.ConstraintApplicationResult; import io.trino.spi.connector.JoinApplicationResult; @@ -516,20 +517,37 @@ public ConnectorTableProperties getTableProperties(ConnectorSession session, Con return new ConnectorTableProperties(); } + @Override + public ConnectorTableSchema getTableSchema(ConnectorSession session, ConnectorTableHandle table) + { + JdbcTableHandle handle = (JdbcTableHandle) table; + + return new ConnectorTableSchema( + getSchemaTableName(handle), + jdbcClient.getColumns(session, handle).stream() + .map(JdbcColumnHandle::getColumnSchema) + .collect(toImmutableList())); + } + @Override public ConnectorTableMetadata getTableMetadata(ConnectorSession session, ConnectorTableHandle table) { JdbcTableHandle handle = (JdbcTableHandle) table; - ImmutableList.Builder columnMetadata = ImmutableList.builder(); - for (JdbcColumnHandle column : jdbcClient.getColumns(session, handle)) { - columnMetadata.add(column.getColumnMetadata()); - } - SchemaTableName schemaTableName = handle.isNamedRelation() + return new ConnectorTableMetadata( + getSchemaTableName(handle), + jdbcClient.getColumns(session, handle).stream() + .map(JdbcColumnHandle::getColumnMetadata) + .collect(toImmutableList()), + jdbcClient.getTableProperties(session, handle)); + } + + public static SchemaTableName getSchemaTableName(JdbcTableHandle handle) + { + return handle.isNamedRelation() ? handle.getRequiredNamedRelation().getSchemaTableName() // TODO (https://github.com/trinodb/trino/issues/6694) SchemaTableName should not be required for synthetic ConnectorTableHandle - : new SchemaTableName("_prepared", "query"); - return new ConnectorTableMetadata(schemaTableName, columnMetadata.build(), jdbcClient.getTableProperties(session, handle)); + : new SchemaTableName("_generated", "_generated_query"); } @Override diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcMetadataConfig.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcMetadataConfig.java index 5cbd1a2f1df2..963f5ea624ff 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcMetadataConfig.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcMetadataConfig.java @@ -58,7 +58,8 @@ public boolean isJoinPushdownEnabled() return joinPushdownEnabled; } - @Config("experimental.join-pushdown.enabled") + @LegacyConfig("experimental.join-pushdown.enabled") + @Config("join-pushdown.enabled") @ConfigDescription("Enable join pushdown") public JdbcMetadataConfig setJoinPushdownEnabled(boolean joinPushdownEnabled) { diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcMetadataFactory.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcMetadataFactory.java index 2312e08152d2..192dafd9aa0c 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcMetadataFactory.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcMetadataFactory.java @@ -17,9 +17,10 @@ import javax.inject.Inject; -import java.util.concurrent.TimeUnit; +import java.util.Set; import static java.util.Objects.requireNonNull; +import static java.util.concurrent.TimeUnit.DAYS; public class JdbcMetadataFactory { @@ -36,6 +37,13 @@ public JdbcMetadataFactory(JdbcClient jdbcClient, JdbcMetadataConfig config) public JdbcMetadata create() { - return new JdbcMetadata(new TransactionCachingJdbcClient(jdbcClient, new Duration(1, TimeUnit.DAYS)), allowDropTable); + // Session stays the same per transaction, therefore session properties don't need to + // be a part of cache keys in CachingJdbcClient. + return new JdbcMetadata(new CachingJdbcClient( + jdbcClient, + Set.of(), + new SingletonJdbcIdentityCacheMapping(), + new Duration(1, DAYS), true), + allowDropTable); } } diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcModule.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcModule.java index f68020ec4399..e737de87cfdd 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcModule.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcModule.java @@ -13,6 +13,7 @@ */ package io.trino.plugin.jdbc; +import com.google.common.util.concurrent.MoreExecutors; import com.google.inject.Binder; import com.google.inject.Key; import com.google.inject.Module; @@ -27,9 +28,12 @@ import io.trino.spi.connector.ConnectorSplitManager; import io.trino.spi.procedure.Procedure; +import javax.annotation.PreDestroy; import javax.inject.Provider; import javax.inject.Singleton; +import java.util.concurrent.ExecutorService; + import static com.google.inject.multibindings.Multibinder.newSetBinder; import static com.google.inject.multibindings.OptionalBinder.newOptionalBinder; import static io.airlift.configuration.ConfigBinder.configBinder; @@ -71,6 +75,11 @@ public void configure(Binder binder) binder.bind(JdbcClient.class).to(CachingJdbcClient.class).in(Scopes.SINGLETON); newOptionalBinder(binder, Key.get(int.class, MaxDomainCompactionThreshold.class)); + + newOptionalBinder(binder, Key.get(ExecutorService.class, ForRecordCursor.class)) + .setDefault() + .toProvider(MoreExecutors::newDirectExecutorService) + .in(Scopes.SINGLETON); } @Provides @@ -109,4 +118,10 @@ public static void bindTablePropertiesProvider(Binder binder, Class getConnectorFactories() { - return ImmutableList.of(new JdbcConnectorFactory(name, combine(new CredentialProviderModule(), module))); + return ImmutableList.of(new JdbcConnectorFactory( + name, + combine( + new CredentialProviderModule(), + new ExtraCredentialsBasedJdbcIdentityCacheMappingModule(), + module))); } } diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcRecordCursor.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcRecordCursor.java index 83076dde34d5..aa1a94b1a793 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcRecordCursor.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcRecordCursor.java @@ -29,6 +29,9 @@ import java.sql.SQLException; import java.sql.Statement; import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; @@ -41,6 +44,8 @@ public class JdbcRecordCursor { private static final Logger log = Logger.get(JdbcRecordCursor.class); + private final ExecutorService executor; + private final JdbcColumnHandle[] columnHandles; private final ReadFunction[] readFunctions; private final BooleanReadFunction[] booleanReadFunctions; @@ -56,9 +61,10 @@ public class JdbcRecordCursor private ResultSet resultSet; private boolean closed; - public JdbcRecordCursor(JdbcClient jdbcClient, ConnectorSession session, JdbcSplit split, JdbcTableHandle table, List columnHandles) + public JdbcRecordCursor(JdbcClient jdbcClient, ExecutorService executor, ConnectorSession session, JdbcSplit split, JdbcTableHandle table, List columnHandles) { this.jdbcClient = requireNonNull(jdbcClient, "jdbcClient is null"); + this.executor = requireNonNull(executor, "executor is null"); this.columnHandles = columnHandles.toArray(new JdbcColumnHandle[0]); @@ -135,8 +141,31 @@ public boolean advanceNextPosition() try { if (resultSet == null) { - log.debug("Executing: %s", statement.toString()); - resultSet = statement.executeQuery(); + Future resultSetFuture = executor.submit(() -> { + log.debug("Executing: %s", statement.toString()); + return statement.executeQuery(); + }); + try { + // statement.executeQuery() may block uninterruptedly, using async way so we are able to cancel remote query + // See javadoc of java.sql.Connection.setNetworkTimeout + resultSet = resultSetFuture.get(); + } + catch (ExecutionException e) { + if (e.getCause() instanceof SQLException) { + SQLException cause = (SQLException) e.getCause(); + SQLException sqlException = new SQLException(cause.getMessage(), cause.getSQLState(), cause.getErrorCode(), e); + if (cause.getNextException() != null) { + sqlException.setNextException(cause.getNextException()); + } + throw sqlException; + } + throw new RuntimeException(e); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + resultSetFuture.cancel(true); + throw new RuntimeException(e); + } } return resultSet.next(); } diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcRecordSet.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcRecordSet.java index b9830da50d3d..40034364c5c4 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcRecordSet.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcRecordSet.java @@ -20,6 +20,7 @@ import io.trino.spi.type.Type; import java.util.List; +import java.util.concurrent.ExecutorService; import static java.util.Objects.requireNonNull; @@ -27,15 +28,17 @@ public class JdbcRecordSet implements RecordSet { private final JdbcClient jdbcClient; + private final ExecutorService executor; private final JdbcTableHandle table; private final List columnHandles; private final List columnTypes; private final JdbcSplit split; private final ConnectorSession session; - public JdbcRecordSet(JdbcClient jdbcClient, ConnectorSession session, JdbcSplit split, JdbcTableHandle table, List columnHandles) + public JdbcRecordSet(JdbcClient jdbcClient, ExecutorService executor, ConnectorSession session, JdbcSplit split, JdbcTableHandle table, List columnHandles) { this.jdbcClient = requireNonNull(jdbcClient, "jdbcClient is null"); + this.executor = requireNonNull(executor, "executor is null"); this.split = requireNonNull(split, "split is null"); this.table = requireNonNull(table, "table is null"); @@ -57,6 +60,6 @@ public List getColumnTypes() @Override public RecordCursor cursor() { - return new JdbcRecordCursor(jdbcClient, session, split, table, columnHandles); + return new JdbcRecordCursor(jdbcClient, executor, session, split, table, columnHandles); } } diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcRecordSetProvider.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcRecordSetProvider.java index e5c48fa07934..03b80ac6f509 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcRecordSetProvider.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcRecordSetProvider.java @@ -25,6 +25,7 @@ import javax.inject.Inject; import java.util.List; +import java.util.concurrent.ExecutorService; import static com.google.common.base.Verify.verify; import static java.util.Objects.requireNonNull; @@ -33,11 +34,13 @@ public class JdbcRecordSetProvider implements ConnectorRecordSetProvider { private final JdbcClient jdbcClient; + private final ExecutorService executor; @Inject - public JdbcRecordSetProvider(JdbcClient jdbcClient) + public JdbcRecordSetProvider(JdbcClient jdbcClient, @ForRecordCursor ExecutorService executor) { this.jdbcClient = requireNonNull(jdbcClient, "jdbcClient is null"); + this.executor = requireNonNull(executor, "executor is null"); } @Override @@ -58,6 +61,6 @@ public RecordSet getRecordSet(ConnectorTransactionHandle transaction, ConnectorS handles.add((JdbcColumnHandle) handle); } - return new JdbcRecordSet(jdbcClient, session, jdbcSplit, jdbcTable, handles.build()); + return new JdbcRecordSet(jdbcClient, executor, session, jdbcSplit, jdbcTable, handles.build()); } } diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcSortItem.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcSortItem.java index f33e45f85c80..6ce11f1b0c4e 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcSortItem.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcSortItem.java @@ -67,4 +67,14 @@ public int hashCode() { return Objects.hash(column, sortOrder); } + + @Override + public String toString() + { + return new StringBuilder() + .append(column) + .append(" ") + .append(sortOrder) + .toString(); + } } diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/RemoteQueryCancellationConfig.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/RemoteQueryCancellationConfig.java new file mode 100644 index 000000000000..ae9b3fde77f8 --- /dev/null +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/RemoteQueryCancellationConfig.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 io.trino.plugin.jdbc; + +import io.airlift.configuration.Config; +import io.airlift.configuration.ConfigDescription; + +public class RemoteQueryCancellationConfig +{ + private boolean remoteQueryCancellationEnabled = true; + + public boolean isRemoteQueryCancellationEnabled() + { + return remoteQueryCancellationEnabled; + } + + @Config("remote-query-async-cancellation.enabled") + @ConfigDescription("Enable asynchronous remote query cancellation") + public RemoteQueryCancellationConfig setRemoteQueryCancellationEnabled(boolean remoteQueryCancellationEnabled) + { + this.remoteQueryCancellationEnabled = remoteQueryCancellationEnabled; + return this; + } +} diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/RemoteQueryCancellationModule.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/RemoteQueryCancellationModule.java new file mode 100644 index 000000000000..cef4eefc79d0 --- /dev/null +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/RemoteQueryCancellationModule.java @@ -0,0 +1,74 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.jdbc; + +import com.google.inject.Binder; +import com.google.inject.Key; +import com.google.inject.Module; +import com.google.inject.Scopes; +import io.airlift.configuration.AbstractConfigurationAwareModule; +import io.trino.plugin.base.CatalogName; + +import javax.inject.Inject; +import javax.inject.Provider; + +import java.util.concurrent.ExecutorService; + +import static com.google.inject.multibindings.OptionalBinder.newOptionalBinder; +import static io.airlift.concurrent.Threads.daemonThreadsNamed; +import static io.airlift.configuration.ConditionalModule.installModuleIf; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; +import static java.util.concurrent.Executors.newCachedThreadPool; + +public class RemoteQueryCancellationModule + extends AbstractConfigurationAwareModule +{ + @Override + protected void setup(Binder binder) + { + install(installModuleIf( + RemoteQueryCancellationConfig.class, + RemoteQueryCancellationConfig::isRemoteQueryCancellationEnabled, + bindForRecordCursor())); + } + + private static Module bindForRecordCursor() + { + return binder -> { + newOptionalBinder(binder, Key.get(ExecutorService.class, ForRecordCursor.class)) + .setBinding() + .toProvider(RecordCursorExecutorServiceProvider.class) + .in(Scopes.SINGLETON); + }; + } + + private static class RecordCursorExecutorServiceProvider + implements Provider + { + private final CatalogName catalogName; + + @Inject + public RecordCursorExecutorServiceProvider(CatalogName catalogName) + { + this.catalogName = requireNonNull(catalogName, "catalogName is null"); + } + + @Override + public ExecutorService get() + { + return newCachedThreadPool(daemonThreadsNamed(format("%s-record-cursor-%%d", catalogName))); + } + } +} diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/SingletonJdbcIdentityCacheMapping.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/SingletonJdbcIdentityCacheMapping.java new file mode 100644 index 000000000000..5a89060ce007 --- /dev/null +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/SingletonJdbcIdentityCacheMapping.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 io.trino.plugin.jdbc; + +public final class SingletonJdbcIdentityCacheMapping + implements JdbcIdentityCacheMapping +{ + @Override + public JdbcIdentityCacheKey getRemoteUserCacheKey(JdbcIdentity identity) + { + return SingletonJdbcIdentityCacheKey.INSTANCE; + } + + private static final class SingletonJdbcIdentityCacheKey + extends JdbcIdentityCacheKey + { + private static final SingletonJdbcIdentityCacheKey INSTANCE = new SingletonJdbcIdentityCacheKey(); + + @Override + public int hashCode() + { + return 0; + } + + @Override + public boolean equals(Object obj) + { + return obj instanceof SingletonJdbcIdentityCacheMapping; + } + } +} diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/StandardColumnMappings.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/StandardColumnMappings.java index 41c6a5e1d20f..fb4d8ec21d65 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/StandardColumnMappings.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/StandardColumnMappings.java @@ -501,7 +501,7 @@ public static LongReadFunction timestampReadFunction(TimestampType timestampType private static ObjectReadFunction longTimestampReadFunction(TimestampType timestampType) { - checkArgument(timestampType.getPrecision() > TimestampType.MAX_SHORT_PRECISION && timestampType.getPrecision() < MAX_LOCAL_DATE_TIME_PRECISION, + checkArgument(timestampType.getPrecision() > TimestampType.MAX_SHORT_PRECISION && timestampType.getPrecision() <= MAX_LOCAL_DATE_TIME_PRECISION, "Precision is out of range: %s", timestampType.getPrecision()); return ObjectReadFunction.of( LongTimestamp.class, diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/TransactionCachingJdbcClient.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/TransactionCachingJdbcClient.java deleted file mode 100644 index 1e1c88f5487c..000000000000 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/TransactionCachingJdbcClient.java +++ /dev/null @@ -1,91 +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 io.trino.plugin.jdbc; - -import com.google.common.cache.Cache; -import com.google.common.cache.CacheBuilder; -import io.airlift.units.Duration; -import io.trino.spi.connector.ColumnHandle; -import io.trino.spi.connector.ConnectorSession; -import io.trino.spi.predicate.TupleDomain; -import io.trino.spi.statistics.TableStatistics; - -import java.util.Objects; -import java.util.Set; - -import static java.util.Objects.requireNonNull; -import static java.util.concurrent.TimeUnit.MILLISECONDS; - -public class TransactionCachingJdbcClient - extends CachingJdbcClient -{ - private final Cache statisticsCache; - - public TransactionCachingJdbcClient(JdbcClient delegate, Duration cachingTtl) - { - // session stays the same per transaction, therefore session properties don't need to - // be a part of cache keys in CachingJdbcClient - super(delegate, Set.of(), cachingTtl, true); - this.statisticsCache = CacheBuilder.newBuilder() - .expireAfterWrite(cachingTtl.toMillis(), MILLISECONDS) - .build(); - } - - @Override - public TableStatistics getTableStatistics(ConnectorSession session, JdbcTableHandle handle, TupleDomain tupleDomain) - { - // session stays the same per transaction, therefore it doesn't need to be part of cache key - TableStatisticsCacheKey key = new TableStatisticsCacheKey(handle, tupleDomain); - TableStatistics cachedStatistics = statisticsCache.getIfPresent(key); - if (cachedStatistics != null) { - return cachedStatistics; - } - - TableStatistics statistics = super.getTableStatistics(session, handle, tupleDomain); - statisticsCache.put(key, statistics); - return statistics; - } - - private static final class TableStatisticsCacheKey - { - private final JdbcTableHandle tableHandle; - private final TupleDomain tupleDomain; - - private TableStatisticsCacheKey(JdbcTableHandle tableHandle, TupleDomain tupleDomain) - { - this.tableHandle = requireNonNull(tableHandle, "tableHandle is null"); - this.tupleDomain = requireNonNull(tupleDomain, "tupleDomain is null"); - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - TableStatisticsCacheKey that = (TableStatisticsCacheKey) o; - return Objects.equals(tableHandle, that.tableHandle) && - Objects.equals(tupleDomain, that.tupleDomain); - } - - @Override - public int hashCode() - { - return Objects.hash(tableHandle, tupleDomain); - } - } -} diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java index 031d01a0cf4c..c663230b534c 100644 --- a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java @@ -13,32 +13,45 @@ */ package io.trino.plugin.jdbc; +import io.airlift.units.Duration; import io.trino.Session; +import io.trino.spi.QueryId; import io.trino.spi.connector.JoinCondition; import io.trino.spi.connector.SortOrder; import io.trino.sql.planner.assertions.PlanMatchPattern; import io.trino.sql.planner.plan.ExchangeNode; import io.trino.sql.planner.plan.JoinNode; +import io.trino.sql.planner.plan.ProjectNode; import io.trino.sql.planner.plan.TableScanNode; import io.trino.sql.planner.plan.TopNNode; import io.trino.sql.query.QueryAssertions.QueryAssert; import io.trino.testing.BaseConnectorTest; +import io.trino.testing.MaterializedResult; import io.trino.testing.TestingConnectorBehavior; import io.trino.testing.sql.TestTable; +import io.trino.testing.sql.TestView; import org.intellij.lang.annotations.Language; import org.testng.SkipException; +import org.testng.annotations.AfterClass; import org.testng.annotations.Test; import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; import java.util.stream.Stream; +import static com.google.common.base.Preconditions.checkState; import static com.google.common.base.Verify.verify; import static com.google.common.collect.ImmutableList.toImmutableList; import static com.google.common.collect.MoreCollectors.toOptional; +import static io.airlift.concurrent.Threads.daemonThreadsNamed; +import static io.trino.plugin.jdbc.RemoteDatabaseEvent.Status.CANCELLED; +import static io.trino.plugin.jdbc.RemoteDatabaseEvent.Status.RUNNING; import static io.trino.sql.planner.assertions.PlanMatchPattern.anyTree; import static io.trino.sql.planner.assertions.PlanMatchPattern.exchange; import static io.trino.sql.planner.assertions.PlanMatchPattern.node; import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_AGGREGATION_PUSHDOWN; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CANCELLATION; import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_JOIN_PUSHDOWN; import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_JOIN_PUSHDOWN_WITH_DISTINCT_FROM; import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_JOIN_PUSHDOWN_WITH_FULL_JOIN; @@ -48,12 +61,25 @@ import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_PREDICATE_PUSHDOWN_WITH_VARCHAR_EQUALITY; import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_PREDICATE_PUSHDOWN_WITH_VARCHAR_INEQUALITY; import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_TOPN_PUSHDOWN; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_TOPN_PUSHDOWN_WITH_VARCHAR; +import static io.trino.testing.assertions.Assert.assertEventually; import static java.lang.String.format; +import static java.util.Locale.ENGLISH; +import static java.util.concurrent.Executors.newCachedThreadPool; +import static java.util.concurrent.TimeUnit.MINUTES; import static org.assertj.core.api.Assertions.assertThat; public abstract class BaseJdbcConnectorTest extends BaseConnectorTest { + private final ExecutorService executor = newCachedThreadPool(daemonThreadsNamed(getClass().getName())); + + @AfterClass(alwaysRun = true) + public void afterClass() + { + executor.shutdownNow(); + } + @Override protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior) { @@ -170,6 +196,18 @@ public void testTopNPushdown() .ordered() .isFullyPushedDown(); } + + // TopN over LEFT join (enforces SINGLE TopN cannot be pushed below OUTER side of join) + // We expect PARTIAL TopN on the LEFT side of join to be pushed down. + assertThat(query("SELECT * " + + "FROM nation n LEFT JOIN region r ON n.regionkey = r.regionkey " + + "ORDER BY n.nationkey LIMIT 3")) + .ordered() + .isNotFullyPushedDown( + node(TopNNode.class, // FINAL TopN + anyTree(node(JoinNode.class, + node(ExchangeNode.class, node(ProjectNode.class, node(TableScanNode.class))), // no PARTIAL TopN + anyTree(node(TableScanNode.class)))))); } @Test @@ -214,7 +252,7 @@ public void testCaseSensitiveTopNPushdown() } // topN over varchar/char columns should only be pushed down if the remote systems's sort order matches Trino - boolean expectTopNPushdown = hasBehavior(SUPPORTS_PREDICATE_PUSHDOWN_WITH_VARCHAR_INEQUALITY); + boolean expectTopNPushdown = hasBehavior(SUPPORTS_TOPN_PUSHDOWN_WITH_VARCHAR); PlanMatchPattern topNOverTableScan = node(TopNNode.class, anyTree(node(TableScanNode.class))); try (TestTable testTable = new TestTable( @@ -553,4 +591,61 @@ protected Session joinPushdownEnabled(Session session) .setCatalogSessionProperty(session.getCatalog().orElseThrow(), "join_pushdown_enabled", "true") .build(); } + + @Test(timeOut = 60_000) + public void testCancellation() + throws Exception + { + if (!hasBehavior(SUPPORTS_CANCELLATION)) { + throw new SkipException("Cancellation is not supported by given connector"); + } + + try (TestView sleepingView = createSleepingView(new Duration(1, MINUTES))) { + String query = "SELECT * FROM " + sleepingView.getName(); + Future future = executor.submit(() -> assertQueryFails(query, "Query killed. Message: Killed by test")); + QueryId queryId = getQueryId(query); + + assertEventually(() -> assertRemoteQueryStatus(sleepingView.getName(), RUNNING)); + assertUpdate(format("CALL system.runtime.kill_query(query_id => '%s', message => '%s')", queryId, "Killed by test")); + future.get(); + assertEventually(() -> assertRemoteQueryStatus(sleepingView.getName(), CANCELLED)); + } + } + + private void assertRemoteQueryStatus(String tableNameToScan, RemoteDatabaseEvent.Status status) + { + String lowerCasedTableName = tableNameToScan.toLowerCase(ENGLISH); + assertThat(getRemoteDatabaseEvents()) + .filteredOn(event -> event.getQuery().toLowerCase(ENGLISH).contains(lowerCasedTableName)) + .map(RemoteDatabaseEvent::getStatus) + .contains(status); + } + + private QueryId getQueryId(String query) + throws Exception + { + for (int i = 0; i < 100; i++) { + MaterializedResult queriesResult = getQueryRunner().execute(format( + "SELECT query_id FROM system.runtime.queries WHERE query = '%s' AND query NOT LIKE '%%system.runtime.queries%%'", + query)); + int rowCount = queriesResult.getRowCount(); + if (rowCount == 0) { + Thread.sleep(100); + continue; + } + checkState(rowCount == 1, "Too many (%s) query ids were found for: %s", rowCount, query); + return new QueryId((String) queriesResult.getOnlyValue()); + } + throw new IllegalStateException("Query id not found for: " + query); + } + + protected List getRemoteDatabaseEvents() + { + throw new UnsupportedOperationException(); + } + + protected TestView createSleepingView(Duration minimalSleepDuration) + { + throw new UnsupportedOperationException(); + } } diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/H2QueryRunner.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/H2QueryRunner.java index 485e73acb13f..b42ea26d1b19 100644 --- a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/H2QueryRunner.java +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/H2QueryRunner.java @@ -14,6 +14,7 @@ package io.trino.plugin.jdbc; import com.google.common.collect.ImmutableList; +import com.google.inject.Module; import io.trino.Session; import io.trino.plugin.tpch.TpchPlugin; import io.trino.testing.DistributedQueryRunner; @@ -44,13 +45,18 @@ public static DistributedQueryRunner createH2QueryRunner(TpchTable... tables) public static DistributedQueryRunner createH2QueryRunner(Iterable> tables) throws Exception - { return createH2QueryRunner(tables, TestingH2JdbcModule.createProperties()); } public static DistributedQueryRunner createH2QueryRunner(Iterable> tables, Map properties) throws Exception + { + return createH2QueryRunner(tables, properties, new TestingH2JdbcModule()); + } + + public static DistributedQueryRunner createH2QueryRunner(Iterable> tables, Map properties, Module module) + throws Exception { DistributedQueryRunner queryRunner = null; try { @@ -61,7 +67,7 @@ public static DistributedQueryRunner createH2QueryRunner(Iterable> createSchema(properties, "tpch"); - queryRunner.installPlugin(new JdbcPlugin("base-jdbc", new TestingH2JdbcModule())); + queryRunner.installPlugin(new JdbcPlugin("base-jdbc", module)); queryRunner.createCatalog("jdbc", "base-jdbc", properties); copyTpchTables(queryRunner, "tpch", TINY_SCHEMA_NAME, createSession(), tables); diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/RemoteDatabaseEvent.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/RemoteDatabaseEvent.java new file mode 100644 index 000000000000..e6e8753c413b --- /dev/null +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/RemoteDatabaseEvent.java @@ -0,0 +1,76 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.jdbc; + +import java.util.Objects; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static java.util.Objects.requireNonNull; + +public class RemoteDatabaseEvent +{ + private final String query; + private final Status status; + + public RemoteDatabaseEvent(String query, Status status) + { + this.query = requireNonNull(query, "query is null"); + this.status = requireNonNull(status, "status is null"); + } + + public String getQuery() + { + return query; + } + + public Status getStatus() + { + return status; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + RemoteDatabaseEvent that = (RemoteDatabaseEvent) o; + return query.equals(that.query) && status == that.status; + } + + @Override + public int hashCode() + { + return Objects.hash(query, status); + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("query", query) + .add("status", status) + .toString(); + } + + public enum Status + { + RUNNING, + CANCELLED, + DONE, + } +} diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestBaseJdbcConfig.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestBaseJdbcConfig.java index 67add96c1240..6f8eb75c5571 100644 --- a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestBaseJdbcConfig.java +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestBaseJdbcConfig.java @@ -39,7 +39,7 @@ public void testDefaults() .setCaseInsensitiveNameMatching(false) .setCaseInsensitiveNameMatchingCacheTtl(new Duration(1, MINUTES)) .setJdbcTypesMappedToVarchar("") - .setMetadataCacheTtl(Duration.valueOf("0m")) + .setMetadataCacheTtl(new Duration(0, MINUTES)) .setCacheMissing(false)); } @@ -60,7 +60,7 @@ public void testExplicitPropertyMappings() .setCaseInsensitiveNameMatching(true) .setCaseInsensitiveNameMatchingCacheTtl(new Duration(1, SECONDS)) .setJdbcTypesMappedToVarchar("mytype, struct_type1") - .setMetadataCacheTtl(Duration.valueOf("1s")) + .setMetadataCacheTtl(new Duration(1, SECONDS)) .setCacheMissing(true); assertFullMapping(properties, expected); diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestCachingJdbcClient.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestCachingJdbcClient.java index 00c933aca237..73c55c7c9241 100644 --- a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestCachingJdbcClient.java +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestCachingJdbcClient.java @@ -19,6 +19,7 @@ import com.google.common.collect.ImmutableSet; import io.airlift.units.Duration; import io.trino.plugin.base.session.SessionPropertiesProvider; +import io.trino.plugin.jdbc.credential.ExtraCredentialConfig; import io.trino.spi.connector.ColumnHandle; import io.trino.spi.connector.ColumnMetadata; import io.trino.spi.connector.ConnectorSession; @@ -26,6 +27,7 @@ import io.trino.spi.connector.SchemaTableName; import io.trino.spi.connector.TableNotFoundException; import io.trino.spi.predicate.TupleDomain; +import io.trino.spi.security.ConnectorIdentity; import io.trino.spi.session.PropertyMetadata; import io.trino.spi.statistics.Estimate; import io.trino.spi.statistics.TableStatistics; @@ -65,7 +67,7 @@ public class TestCachingJdbcClient null, false)); - public static final Set SESSION_PROPERTIES_PROVIDERS = Set.of(() -> PROPERTY_METADATA); + private static final Set SESSION_PROPERTIES_PROVIDERS = Set.of(() -> PROPERTY_METADATA); private static final ConnectorSession SESSION = TestingConnectorSession.builder() .setPropertyMetadata(PROPERTY_METADATA) @@ -92,7 +94,7 @@ public void setUp() private CachingJdbcClient createCachingJdbcClient(Duration cacheTtl, boolean cacheMissing) { - return new CachingJdbcClient(database.getJdbcClient(), SESSION_PROPERTIES_PROVIDERS, cacheTtl, cacheMissing); + return new CachingJdbcClient(database.getJdbcClient(), SESSION_PROPERTIES_PROVIDERS, new SingletonJdbcIdentityCacheMapping(), cacheTtl, cacheMissing); } private CachingJdbcClient createCachingJdbcClient(boolean cacheMissing) @@ -441,7 +443,7 @@ public TableStatistics getTableStatistics(ConnectorSession session, JdbcTableHan return NON_EMPTY_STATS; } }; - return new CachingJdbcClient(statsAwareJdbcClient, SESSION_PROPERTIES_PROVIDERS, duration, cacheMissing); + return new CachingJdbcClient(statsAwareJdbcClient, SESSION_PROPERTIES_PROVIDERS, new SingletonJdbcIdentityCacheMapping(), duration, cacheMissing); } @Test @@ -482,6 +484,36 @@ public void testGetTableStatisticsDoNotCacheEmptyWhenCachingMissingIsDisabled() this.jdbcClient.dropTable(SESSION, table); } + @Test + public void testDifferentIdentityKeys() + { + CachingJdbcClient cachingJdbcClient = new CachingJdbcClient( + database.getJdbcClient(), + SESSION_PROPERTIES_PROVIDERS, + new ExtraCredentialsBasedJdbcIdentityCacheMapping(new ExtraCredentialConfig() + .setUserCredentialName("user") + .setPasswordCredentialName("password")), + FOREVER, + true); + ConnectorSession alice = createUserSession("alice"); + ConnectorSession bob = createUserSession("bob"); + + JdbcTableHandle table = createTable(new SchemaTableName(schema, "table")); + + assertTableNamesCache(cachingJdbcClient).loads(2).misses(2).afterRunning(() -> { + assertThat(cachingJdbcClient.getTableNames(alice, Optional.empty())).contains(table.getRequiredNamedRelation().getSchemaTableName()); + assertThat(cachingJdbcClient.getTableNames(bob, Optional.empty())).contains(table.getRequiredNamedRelation().getSchemaTableName()); + }); + + assertTableNamesCache(cachingJdbcClient).hits(2).afterRunning(() -> { + assertThat(cachingJdbcClient.getTableNames(alice, Optional.empty())).contains(table.getRequiredNamedRelation().getSchemaTableName()); + assertThat(cachingJdbcClient.getTableNames(bob, Optional.empty())).contains(table.getRequiredNamedRelation().getSchemaTableName()); + }); + + // Drop tables by not using caching jdbc client + jdbcClient.dropTable(SESSION, table); + } + private JdbcTableHandle getAnyTable(String schema) { SchemaTableName tableName = jdbcClient.getTableNames(SESSION, Optional.of(schema)) @@ -516,6 +548,15 @@ private static ConnectorSession createSession(String sessionName) .build(); } + private static ConnectorSession createUserSession(String userName) + { + return builder() + .setIdentity(ConnectorIdentity.forUser(userName) + .withExtraCredentials(ImmutableMap.of("user", userName)) + .build()) + .build(); + } + @Test public void testEverythingImplemented() { @@ -534,6 +575,11 @@ private static Set nonOverriddenMethods() } } + private static CacheStatsAssertions assertTableNamesCache(CachingJdbcClient cachingJdbcClient) + { + return new CacheStatsAssertions(cachingJdbcClient::getTableNamesCacheStats); + } + private static CacheStatsAssertions assertColumnCacheStats(CachingJdbcClient client) { return new CacheStatsAssertions(client::getColumnsCacheStats); diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcClient.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcClient.java index 4fd17577a620..052eb8fcd7bc 100644 --- a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcClient.java +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcClient.java @@ -28,10 +28,14 @@ import static io.trino.plugin.jdbc.TestingJdbcTypeHandle.JDBC_BIGINT; import static io.trino.plugin.jdbc.TestingJdbcTypeHandle.JDBC_DOUBLE; import static io.trino.plugin.jdbc.TestingJdbcTypeHandle.JDBC_REAL; +import static io.trino.plugin.jdbc.TestingJdbcTypeHandle.JDBC_TIMESTAMP; import static io.trino.plugin.jdbc.TestingJdbcTypeHandle.JDBC_VARCHAR; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.spi.type.DoubleType.DOUBLE; import static io.trino.spi.type.RealType.REAL; +import static io.trino.spi.type.TimestampType.TIMESTAMP_MICROS; +import static io.trino.spi.type.TimestampType.TIMESTAMP_MILLIS; +import static io.trino.spi.type.TimestampType.TIMESTAMP_NANOS; import static io.trino.spi.type.VarcharType.VARCHAR; import static io.trino.spi.type.VarcharType.createVarcharType; import static io.trino.testing.TestingSession.testSessionBuilder; @@ -70,6 +74,7 @@ public void testMetadata() assertTrue(jdbcClient.getSchemaNames(session).containsAll(ImmutableSet.of("example", "tpch"))); assertEquals(jdbcClient.getTableNames(session, Optional.of("example")), ImmutableList.of( new SchemaTableName("example", "numbers"), + new SchemaTableName("example", "timestamps"), new SchemaTableName("example", "view_source"), new SchemaTableName("example", "view"))); assertEquals(jdbcClient.getTableNames(session, Optional.of("tpch")), ImmutableList.of( @@ -113,6 +118,18 @@ public void testMetadataWithFloatAndDoubleCol() new JdbcColumnHandle("COL4", JDBC_REAL, REAL))); } + @Test + public void testMetadataWithTimestampCol() + { + SchemaTableName schemaTableName = new SchemaTableName("example", "timestamps"); + Optional table = jdbcClient.getTableHandle(session, schemaTableName); + assertTrue(table.isPresent(), "table is missing"); + assertEquals(jdbcClient.getColumns(session, table.get()), ImmutableList.of( + new JdbcColumnHandle("TS_3", JDBC_TIMESTAMP, TIMESTAMP_MILLIS), + new JdbcColumnHandle("TS_6", JDBC_TIMESTAMP, TIMESTAMP_MICROS), + new JdbcColumnHandle("TS_9", JDBC_TIMESTAMP, TIMESTAMP_NANOS))); + } + @Test public void testCreateSchema() { diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcConnectorFactory.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcConnectorFactory.java index 203cf9abb746..952c0ebd7a17 100644 --- a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcConnectorFactory.java +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcConnectorFactory.java @@ -25,7 +25,12 @@ public class TestJdbcConnectorFactory @Test public void test() { - ConnectorFactory connectorFactory = new JdbcConnectorFactory("test", combine(new CredentialProviderModule(), new TestingH2JdbcModule())); + ConnectorFactory connectorFactory = new JdbcConnectorFactory( + "test", + combine( + new CredentialProviderModule(), + new ExtraCredentialsBasedJdbcIdentityCacheMappingModule(), + new TestingH2JdbcModule())); connectorFactory.create("test", TestingH2JdbcModule.createProperties(), new TestingConnectorContext()); } diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcMetadata.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcMetadata.java index f1e5a8b81c3e..2ce592236c80 100644 --- a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcMetadata.java +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcMetadata.java @@ -155,6 +155,7 @@ public void testListTables() // all schemas assertEquals(ImmutableSet.copyOf(metadata.listTables(SESSION, Optional.empty())), ImmutableSet.of( new SchemaTableName("example", "numbers"), + new SchemaTableName("example", "timestamps"), new SchemaTableName("example", "view_source"), new SchemaTableName("example", "view"), new SchemaTableName("tpch", "orders"), @@ -165,6 +166,7 @@ public void testListTables() // specific schema assertEquals(ImmutableSet.copyOf(metadata.listTables(SESSION, Optional.of("example"))), ImmutableSet.of( new SchemaTableName("example", "numbers"), + new SchemaTableName("example", "timestamps"), new SchemaTableName("example", "view_source"), new SchemaTableName("example", "view"))); assertEquals(ImmutableSet.copyOf(metadata.listTables(SESSION, Optional.of("tpch"))), ImmutableSet.of( diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcMetadataConfig.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcMetadataConfig.java index 7eac23b26b0c..693136be110d 100644 --- a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcMetadataConfig.java +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcMetadataConfig.java @@ -40,7 +40,7 @@ public void testExplicitPropertyMappings() { Map properties = new ImmutableMap.Builder() .put("allow-drop-table", "true") - .put("experimental.join-pushdown.enabled", "true") + .put("join-pushdown.enabled", "true") .put("aggregation-pushdown.enabled", "false") .put("domain-compaction-threshold", "42") .put("topn-pushdown.enabled", "false") diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcRecordSet.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcRecordSet.java index 01da3d85ad74..9ae03708e69b 100644 --- a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcRecordSet.java +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcRecordSet.java @@ -25,7 +25,10 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.ExecutorService; +import static com.google.common.util.concurrent.MoreExecutors.newDirectExecutorService; +import static io.airlift.testing.Closeables.closeAll; import static io.trino.plugin.jdbc.TestingJdbcTypeHandle.JDBC_BIGINT; import static io.trino.plugin.jdbc.TestingJdbcTypeHandle.JDBC_VARCHAR; import static io.trino.spi.type.BigintType.BIGINT; @@ -42,6 +45,7 @@ public class TestJdbcRecordSet private JdbcTableHandle table; private JdbcSplit split; private Map columnHandles; + private ExecutorService executor; @BeforeClass public void setUp() @@ -52,13 +56,16 @@ public void setUp() table = database.getTableHandle(SESSION, new SchemaTableName("example", "numbers")); split = database.getSplit(SESSION, table); columnHandles = database.getColumnHandles(SESSION, table); + executor = newDirectExecutorService(); } @AfterClass(alwaysRun = true) public void tearDown() throws Exception { - database.close(); + closeAll( + database, + () -> executor.shutdownNow()); } @Test @@ -163,6 +170,6 @@ public void testIdempotentClose() private JdbcRecordSet createRecordSet(List columnHandles) { - return new JdbcRecordSet(jdbcClient, SESSION, split, table, columnHandles); + return new JdbcRecordSet(jdbcClient, executor, SESSION, split, table, columnHandles); } } diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcRecordSetProvider.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcRecordSetProvider.java index 42dc1be2ba9f..8f0bddd5d312 100644 --- a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcRecordSetProvider.java +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcRecordSetProvider.java @@ -36,10 +36,13 @@ import java.util.Map; import java.util.Optional; import java.util.OptionalLong; +import java.util.concurrent.ExecutorService; import static com.google.common.collect.Iterables.getOnlyElement; +import static com.google.common.util.concurrent.MoreExecutors.newDirectExecutorService; import static io.airlift.concurrent.MoreFutures.getFutureValue; import static io.airlift.slice.Slices.utf8Slice; +import static io.airlift.testing.Closeables.closeAll; import static io.trino.spi.connector.NotPartitionedPartitionHandle.NOT_PARTITIONED; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.spi.type.VarcharType.VARCHAR; @@ -62,6 +65,8 @@ public class TestJdbcRecordSetProvider private JdbcColumnHandle textShortColumn; private JdbcColumnHandle valueColumn; + private ExecutorService executor; + @BeforeClass public void setUp() throws Exception @@ -75,20 +80,24 @@ public void setUp() textColumn = columns.get("text"); textShortColumn = columns.get("text_short"); valueColumn = columns.get("value"); + + executor = newDirectExecutorService(); } @AfterClass(alwaysRun = true) public void tearDown() throws Exception { - database.close(); + closeAll( + database, + () -> executor.shutdownNow()); } @Test public void testGetRecordSet() { ConnectorTransactionHandle transaction = new JdbcTransactionHandle(); - JdbcRecordSetProvider recordSetProvider = new JdbcRecordSetProvider(jdbcClient); + JdbcRecordSetProvider recordSetProvider = new JdbcRecordSetProvider(jdbcClient, executor); RecordSet recordSet = recordSetProvider.getRecordSet(transaction, SESSION, split, table, ImmutableList.of(textColumn, textShortColumn, valueColumn)); assertNotNull(recordSet, "recordSet is null"); @@ -196,7 +205,7 @@ private RecordCursor getCursor(JdbcTableHandle jdbcTableHandle, List properties = TestingH2JdbcModule.createProperties(); + private Runnable onGetTableProperties = () -> {}; + + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + TestingH2JdbcModule module = new TestingH2JdbcModule((config, connectionFactory) -> new TestingH2JdbcClient(config, connectionFactory) + { + @Override + public Map getTableProperties(ConnectorSession session, JdbcTableHandle tableHandle) + { + onGetTableProperties.run(); + return ImmutableMap.of(); + } + }); + return createH2QueryRunner(ImmutableList.copyOf(TpchTable.getTables()), properties, module); + } + + @BeforeTest + public void reset() + { + onGetTableProperties = () -> {}; + } + + @Test + public void testGetTablePropertiesIsNotCalledForSelect() + { + onGetTableProperties = () -> { fail("Unexpected call of: getTableProperties"); }; + assertUpdate("CREATE TABLE copy_of_nation AS SELECT * FROM nation", 25); + assertQuerySucceeds("SELECT * FROM copy_of_nation"); + assertQuerySucceeds("SELECT nationkey FROM copy_of_nation"); + } + + @Test + public void testGetTablePropertiesIsCalled() + { + AtomicInteger counter = new AtomicInteger(); + onGetTableProperties = () -> { counter.incrementAndGet(); }; + assertQuerySucceeds("SHOW CREATE TABLE nation"); + assertThat(counter.get()).isOne(); + } +} diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestRemoteQueryCancellationConfig.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestRemoteQueryCancellationConfig.java new file mode 100644 index 000000000000..7964ecf1a8fb --- /dev/null +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestRemoteQueryCancellationConfig.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 io.trino.plugin.jdbc; + +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 TestRemoteQueryCancellationConfig +{ + @Test + public void testDefaults() + { + assertRecordedDefaults(recordDefaults(RemoteQueryCancellationConfig.class) + .setRemoteQueryCancellationEnabled(true)); + } + + @Test + public void testExplicitPropertyMappings() + { + Map properties = new ImmutableMap.Builder() + .put("remote-query-async-cancellation.enabled", "false") + .build(); + + RemoteQueryCancellationConfig expected = new RemoteQueryCancellationConfig() + .setRemoteQueryCancellationEnabled(false); + + assertFullMapping(properties, expected); + } +} diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestingDatabase.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestingDatabase.java index 194e7460f44e..163fe9dc6e9b 100644 --- a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestingDatabase.java +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestingDatabase.java @@ -52,6 +52,7 @@ public TestingDatabase() connection.createStatement().execute("CREATE SCHEMA example"); connection.createStatement().execute("CREATE TABLE example.numbers(text varchar primary key, text_short varchar(32), value bigint)"); + connection.createStatement().execute("CREATE TABLE example.timestamps(ts_3 timestamp(3) primary key, ts_6 timestamp(6), ts_9 timestamp(9))"); connection.createStatement().execute("INSERT INTO example.numbers(text, text_short, value) VALUES " + "('one', 'one', 1)," + "('two', 'two', 2)," + diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestingH2JdbcClient.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestingH2JdbcClient.java index f45911a49017..4a75e13a5228 100644 --- a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestingH2JdbcClient.java +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestingH2JdbcClient.java @@ -21,6 +21,7 @@ import io.trino.spi.connector.ColumnHandle; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.SchemaTableName; +import io.trino.spi.type.TimestampType; import io.trino.spi.type.Type; import io.trino.spi.type.VarcharType; @@ -130,7 +131,10 @@ public Optional toColumnMapping(ConnectorSession session, Connect return Optional.of(timeColumnMapping(TIME_MILLIS)); case Types.TIMESTAMP: - return Optional.of(timestampColumnMapping(TIMESTAMP_MILLIS)); + TimestampType timestampType = typeHandle.getDecimalDigits() + .map(TimestampType::createTimestampType) + .orElse(TIMESTAMP_MILLIS); + return Optional.of(timestampColumnMapping(timestampType)); } if (getUnsupportedTypeHandling(session) == CONVERT_TO_VARCHAR) { diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestingH2JdbcModule.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestingH2JdbcModule.java index 70bfa71b0d74..ffe2b5904d95 100644 --- a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestingH2JdbcModule.java +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestingH2JdbcModule.java @@ -25,10 +25,23 @@ import java.util.concurrent.ThreadLocalRandom; import static java.lang.String.format; +import static java.util.Objects.requireNonNull; public class TestingH2JdbcModule implements Module { + private final TestingH2JdbcClientFactory testingH2JdbcClientFactory; + + public TestingH2JdbcModule() + { + this((config, connectionFactory) -> new TestingH2JdbcClient(config, connectionFactory)); + } + + public TestingH2JdbcModule(TestingH2JdbcClientFactory testingH2JdbcClientFactory) + { + this.testingH2JdbcClientFactory = requireNonNull(testingH2JdbcClientFactory, "testingH2JdbcClientFactory is null"); + } + @Override public void configure(Binder binder) {} @@ -36,7 +49,7 @@ public void configure(Binder binder) {} @ForBaseJdbc public JdbcClient provideJdbcClient(BaseJdbcConfig config, ConnectionFactory connectionFactory) { - return new TestingH2JdbcClient(config, connectionFactory); + return testingH2JdbcClientFactory.create(config, connectionFactory); } @Provides @@ -53,4 +66,9 @@ public static Map createProperties() .put("connection-url", format("jdbc:h2:mem:test%s;DB_CLOSE_DELAY=-1", System.nanoTime() + ThreadLocalRandom.current().nextLong())) .build(); } + + public interface TestingH2JdbcClientFactory + { + TestingH2JdbcClient create(BaseJdbcConfig config, ConnectionFactory connectionFactory); + } } diff --git a/plugin/trino-bigquery/pom.xml b/plugin/trino-bigquery/pom.xml index 3abf2cfcc99a..2bf44208b3be 100644 --- a/plugin/trino-bigquery/pom.xml +++ b/plugin/trino-bigquery/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-bigquery/src/main/java/io/trino/plugin/bigquery/BigQueryClient.java b/plugin/trino-bigquery/src/main/java/io/trino/plugin/bigquery/BigQueryClient.java index 65c934ec72de..ded50a1cfa94 100644 --- a/plugin/trino-bigquery/src/main/java/io/trino/plugin/bigquery/BigQueryClient.java +++ b/plugin/trino-bigquery/src/main/java/io/trino/plugin/bigquery/BigQueryClient.java @@ -34,7 +34,6 @@ import io.airlift.units.Duration; import io.trino.spi.TrinoException; import io.trino.spi.connector.TableNotFoundException; -import org.checkerframework.checker.nullness.qual.Nullable; import java.util.Collections; import java.util.HashMap; @@ -89,7 +88,7 @@ Optional toRemoteDataset(String projectId, String datasetN return Optional.of(RemoteDatabaseObject.of(datasetName)); } - @Nullable Optional remoteDataset = remoteDatasets.getIfPresent(datasetName); + Optional remoteDataset = remoteDatasets.getIfPresent(datasetName); if (remoteDataset != null) { return remoteDataset; } @@ -123,7 +122,7 @@ Optional toRemoteTable(String projectId, String remoteData } TableId cacheKey = TableId.of(projectId, remoteDatasetName, tableName); - @Nullable Optional remoteTable = remoteTables.getIfPresent(cacheKey); + Optional remoteTable = remoteTables.getIfPresent(cacheKey); if (remoteTable != null) { return remoteTable; } @@ -264,7 +263,7 @@ List getColumns(BigQueryTableHandle tableHandle) tableHandle.getSchemaTableName(), format("Table '%s' not found", tableHandle.getSchemaTableName())); } - @Nullable Schema schema = tableInfo.getDefinition().getSchema(); + Schema schema = tableInfo.getDefinition().getSchema(); if (schema == null) { throw new TableNotFoundException( tableHandle.getSchemaTableName(), diff --git a/plugin/trino-blackhole/pom.xml b/plugin/trino-blackhole/pom.xml index 6d304095166b..5fad955e198d 100644 --- a/plugin/trino-blackhole/pom.xml +++ b/plugin/trino-blackhole/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-cassandra/pom.xml b/plugin/trino-cassandra/pom.xml index d59a28511493..63a6e51628a1 100644 --- a/plugin/trino-cassandra/pom.xml +++ b/plugin/trino-cassandra/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-clickhouse/pom.xml b/plugin/trino-clickhouse/pom.xml index c913ffa79fe4..1b498e0d41d9 100644 --- a/plugin/trino-clickhouse/pom.xml +++ b/plugin/trino-clickhouse/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-druid/pom.xml b/plugin/trino-druid/pom.xml index f5c2133dbe29..e38bda244f4b 100644 --- a/plugin/trino-druid/pom.xml +++ b/plugin/trino-druid/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-druid/src/test/java/io/trino/plugin/druid/BaseDruidConnectorTest.java b/plugin/trino-druid/src/test/java/io/trino/plugin/druid/BaseDruidConnectorTest.java index 4792fba22544..7b4e4adf09d6 100644 --- a/plugin/trino-druid/src/test/java/io/trino/plugin/druid/BaseDruidConnectorTest.java +++ b/plugin/trino-druid/src/test/java/io/trino/plugin/druid/BaseDruidConnectorTest.java @@ -128,6 +128,7 @@ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior) case SUPPORTS_RENAME_TABLE: case SUPPORTS_COMMENT_ON_COLUMN: case SUPPORTS_COMMENT_ON_TABLE: + case SUPPORTS_TOPN_PUSHDOWN: return false; default: return super.hasBehavior(connectorBehavior); diff --git a/plugin/trino-elasticsearch/pom.xml b/plugin/trino-elasticsearch/pom.xml index 076d5c86bd5e..6bdfae5fa6e1 100644 --- a/plugin/trino-elasticsearch/pom.xml +++ b/plugin/trino-elasticsearch/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-example-http/pom.xml b/plugin/trino-example-http/pom.xml index 397ef59bf1e7..ae20d60f2309 100644 --- a/plugin/trino-example-http/pom.xml +++ b/plugin/trino-example-http/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-geospatial/pom.xml b/plugin/trino-geospatial/pom.xml index 136a1f22882a..d9d6f60cf0a1 100644 --- a/plugin/trino-geospatial/pom.xml +++ b/plugin/trino-geospatial/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-google-sheets/pom.xml b/plugin/trino-google-sheets/pom.xml index ea487e088f86..0aec5ceb3fca 100644 --- a/plugin/trino-google-sheets/pom.xml +++ b/plugin/trino-google-sheets/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-hive-hadoop2/pom.xml b/plugin/trino-hive-hadoop2/pom.xml index bfcd308f7302..16e99dc6c048 100644 --- a/plugin/trino-hive-hadoop2/pom.xml +++ b/plugin/trino-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/TestHiveAlluxioMetastore.java b/plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/TestHiveAlluxioMetastore.java index 6e0677a765f0..569f8c8fcc1b 100644 --- a/plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/TestHiveAlluxioMetastore.java +++ b/plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/TestHiveAlluxioMetastore.java @@ -232,6 +232,12 @@ public void testApplyRedirection() // Alluxio metastore does not support create/delete operations } + @Override + public void testMaterializedViewMetadata() + { + // Alluxio metastore does not support create/delete operations + } + @Override public void testPreferredInsertLayout() { diff --git a/plugin/trino-hive/pom.xml b/plugin/trino-hive/pom.xml index 1073599e470d..8c52068cce87 100644 --- a/plugin/trino-hive/pom.xml +++ b/plugin/trino-hive/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml @@ -187,7 +187,7 @@ com.linkedin.coral - coral-presto + coral-trino diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/AcidInfo.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/AcidInfo.java index 3cd06c037856..d417d9377f78 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/AcidInfo.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/AcidInfo.java @@ -39,18 +39,21 @@ public class AcidInfo private final List deleteDeltas; private final List originalFiles; private final int bucketId; + private final boolean orcAcidVersionValidated; @JsonCreator public AcidInfo( @JsonProperty("partitionLocation") String partitionLocation, @JsonProperty("deleteDeltas") List deleteDeltas, @JsonProperty("originalFiles") List originalFiles, - @JsonProperty("bucketId") int bucketId) + @JsonProperty("bucketId") int bucketId, + @JsonProperty("orcAcidVersionValidated") boolean orcAcidVersionValidated) { this.partitionLocation = requireNonNull(partitionLocation, "partitionLocation is null"); this.deleteDeltas = ImmutableList.copyOf(requireNonNull(deleteDeltas, "deleteDeltas is null")); this.originalFiles = ImmutableList.copyOf(requireNonNull(originalFiles, "originalFiles is null")); this.bucketId = bucketId; + this.orcAcidVersionValidated = orcAcidVersionValidated; } @JsonProperty @@ -77,6 +80,12 @@ public List getDeleteDeltas() return deleteDeltas; } + @JsonProperty + public boolean isOrcAcidVersionValidated() + { + return orcAcidVersionValidated; + } + @Override public boolean equals(Object o) { @@ -88,6 +97,7 @@ public boolean equals(Object o) } AcidInfo that = (AcidInfo) o; return bucketId == that.bucketId && + orcAcidVersionValidated == that.orcAcidVersionValidated && Objects.equals(partitionLocation, that.partitionLocation) && Objects.equals(deleteDeltas, that.deleteDeltas) && Objects.equals(originalFiles, that.originalFiles); @@ -96,7 +106,7 @@ public boolean equals(Object o) @Override public int hashCode() { - return Objects.hash(partitionLocation, deleteDeltas, originalFiles, bucketId); + return Objects.hash(partitionLocation, deleteDeltas, originalFiles, bucketId, orcAcidVersionValidated); } @Override @@ -107,6 +117,7 @@ public String toString() .add("deleteDeltas", deleteDeltas) .add("originalFiles", originalFiles) .add("bucketId", bucketId) + .add("orcAcidVersionValidated", orcAcidVersionValidated) .toString(); } @@ -225,6 +236,7 @@ public static class Builder private final Path partitionLocation; private final List deleteDeltaInfos = new ArrayList<>(); private final ListMultimap bucketIdToOriginalFileInfoMap = ArrayListMultimap.create(); + private boolean orcAcidVersionValidated; private Builder(Path partitionPath) { @@ -265,6 +277,12 @@ public Builder addOriginalFile(Path originalFilePath, long originalFileLength, i return this; } + public Builder setOrcAcidVersionValidated(boolean orcAcidVersionValidated) + { + this.orcAcidVersionValidated = orcAcidVersionValidated; + return this; + } + public AcidInfo buildWithRequiredOriginalFiles(int bucketId) { checkState( @@ -272,16 +290,18 @@ public AcidInfo buildWithRequiredOriginalFiles(int bucketId) "Bucket Id to OriginalFileInfo map should have entry for requested bucket id: %s", bucketId); List deleteDeltas = ImmutableList.copyOf(deleteDeltaInfos); - return new AcidInfo(partitionLocation.toString(), deleteDeltas, bucketIdToOriginalFileInfoMap.get(bucketId), bucketId); + return new AcidInfo(partitionLocation.toString(), deleteDeltas, bucketIdToOriginalFileInfoMap.get(bucketId), bucketId, orcAcidVersionValidated); } public Optional build() { List deleteDeltas = ImmutableList.copyOf(deleteDeltaInfos); - if (deleteDeltas.isEmpty()) { + if (deleteDeltas.isEmpty() && orcAcidVersionValidated) { + // We do not want to bail out with `Optional.empty()` if ORC ACID version was not validated based on _orc_acid_version file. + // If we did so extra validation in OrcPageSourceFactory (based on file metadata) would not be performed. return Optional.empty(); } - return Optional.of(new AcidInfo(partitionLocation.toString(), deleteDeltas, ImmutableList.of(), -1)); + return Optional.of(new AcidInfo(partitionLocation.toString(), deleteDeltas, ImmutableList.of(), -1, orcAcidVersionValidated)); } } } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/BackgroundHiveSplitLoader.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/BackgroundHiveSplitLoader.java index a5d02e504533..328251193a21 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/BackgroundHiveSplitLoader.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/BackgroundHiveSplitLoader.java @@ -497,8 +497,14 @@ private ListenableFuture loadPartition(HivePartitionMetadata partition) ? directory.getBaseDirectory() : (directory.getCurrentDirectories().size() > 0 ? directory.getCurrentDirectories().get(0).getPath() : null); - if (baseOrDeltaPath != null && AcidUtils.OrcAcidVersion.getAcidVersionFromMetaFile(baseOrDeltaPath, fs) < 2) { - throw new TrinoException(NOT_SUPPORTED, "Hive transactional tables are supported with Hive 3.0 and only after a major compaction has been run"); + if (baseOrDeltaPath != null && AcidUtils.OrcAcidVersion.getAcidVersionFromMetaFile(baseOrDeltaPath, fs) >= 2) { + // Trino cannot read ORC ACID tables with version < 2 (written by Hive older than 3.0) + // See https://github.com/trinodb/trino/issues/2790#issuecomment-591901728 for more context + + // We perform initial version check based on _orc_acid_version file here. + // If we cannot verify the version (the _orc_acid_version file may not exist), + // we will do extra check based on ORC datafile metadata in OrcPageSourceFactory. + acidInfoBuilder.setOrcAcidVersionValidated(true); } } @@ -538,6 +544,8 @@ private ListenableFuture loadPartition(HivePartitionMetadata partition) } } else { + // TODO https://github.com/trinodb/trino/issues/7603 - we should not referece acidInfoBuilder at allwhen we are not reading from non-ACID table + acidInfoBuilder.setOrcAcidVersionValidated(true); // no ACID; no further validation needed readPaths = ImmutableList.of(path); } // Bucketed partitions are fully loaded immediately since all files must be loaded to determine the file to bucket mapping diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMaterializedViewMetadata.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMaterializedViewMetadata.java new file mode 100644 index 000000000000..61ea0070196d --- /dev/null +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMaterializedViewMetadata.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 io.trino.plugin.hive; + +import io.trino.spi.connector.ConnectorMaterializedViewDefinition; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.MaterializedViewFreshness; +import io.trino.spi.connector.SchemaTableName; + +import java.util.Optional; + +public interface HiveMaterializedViewMetadata +{ + void createMaterializedView(ConnectorSession session, SchemaTableName viewName, ConnectorMaterializedViewDefinition definition, boolean replace, boolean ignoreExisting); + + void dropMaterializedView(ConnectorSession session, SchemaTableName viewName); + + Optional getMaterializedView(ConnectorSession session, SchemaTableName viewName); + + MaterializedViewFreshness getMaterializedViewFreshness(ConnectorSession session, SchemaTableName name); +} diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadata.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadata.java index a10c640e2228..3ec67ef65f82 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadata.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadata.java @@ -59,6 +59,7 @@ import io.trino.spi.connector.ColumnHandle; import io.trino.spi.connector.ColumnMetadata; import io.trino.spi.connector.ConnectorInsertTableHandle; +import io.trino.spi.connector.ConnectorMaterializedViewDefinition; import io.trino.spi.connector.ConnectorNewTableLayout; import io.trino.spi.connector.ConnectorOutputMetadata; import io.trino.spi.connector.ConnectorOutputTableHandle; @@ -74,6 +75,7 @@ import io.trino.spi.connector.ConstraintApplicationResult; import io.trino.spi.connector.DiscretePredicates; import io.trino.spi.connector.InMemoryRecordSet; +import io.trino.spi.connector.MaterializedViewFreshness; import io.trino.spi.connector.ProjectionApplicationResult; import io.trino.spi.connector.SchemaNotFoundException; import io.trino.spi.connector.SchemaTableName; @@ -322,6 +324,7 @@ public class HiveMetadata private final String prestoVersion; private final HiveStatisticsProvider hiveStatisticsProvider; private final HiveRedirectionsProvider hiveRedirectionsProvider; + private final HiveMaterializedViewMetadata hiveMaterializedViewMetadata; private final AccessControlMetadata accessControlMetadata; public HiveMetadata( @@ -339,6 +342,7 @@ public HiveMetadata( String trinoVersion, HiveStatisticsProvider hiveStatisticsProvider, HiveRedirectionsProvider hiveRedirectionsProvider, + HiveMaterializedViewMetadata hiveMaterializedViewMetadata, AccessControlMetadata accessControlMetadata) { this.catalogName = requireNonNull(catalogName, "catalogName is null"); @@ -355,6 +359,7 @@ public HiveMetadata( this.prestoVersion = requireNonNull(trinoVersion, "trinoVersion is null"); this.hiveStatisticsProvider = requireNonNull(hiveStatisticsProvider, "hiveStatisticsProvider is null"); this.hiveRedirectionsProvider = requireNonNull(hiveRedirectionsProvider, "hiveRedirectionsProvider is null"); + this.hiveMaterializedViewMetadata = requireNonNull(hiveMaterializedViewMetadata, "hiveMaterializedViewMetadata is null"); this.accessControlMetadata = requireNonNull(accessControlMetadata, "accessControlMetadata is null"); } @@ -1573,8 +1578,8 @@ public ConnectorTableHandle beginUpdate(ConnectorSession session, ConnectorTable Table table = metastore.getTable(identity, tableName.getSchemaName(), tableName.getTableName()) .orElseThrow(() -> new TableNotFoundException(tableName)); - if (!isTransactionalTable(table.getParameters())) { - throw new TrinoException(NOT_SUPPORTED, "Hive update is only supported for transactional tables"); + if (!isFullAcidTable(table.getParameters())) { + throw new TrinoException(NOT_SUPPORTED, "Hive update is only supported for ACID transactional tables"); } // Verify that none of the updated columns are partition columns or bucket columns @@ -3002,6 +3007,30 @@ public void cleanupQuery(ConnectorSession session) metastore.cleanupQuery(session); } + @Override + public void createMaterializedView(ConnectorSession session, SchemaTableName viewName, ConnectorMaterializedViewDefinition definition, boolean replace, boolean ignoreExisting) + { + hiveMaterializedViewMetadata.createMaterializedView(session, viewName, definition, replace, ignoreExisting); + } + + @Override + public void dropMaterializedView(ConnectorSession session, SchemaTableName viewName) + { + hiveMaterializedViewMetadata.dropMaterializedView(session, viewName); + } + + @Override + public Optional getMaterializedView(ConnectorSession session, SchemaTableName viewName) + { + return hiveMaterializedViewMetadata.getMaterializedView(session, viewName); + } + + @Override + public MaterializedViewFreshness getMaterializedViewFreshness(ConnectorSession session, SchemaTableName name) + { + return hiveMaterializedViewMetadata.getMaterializedViewFreshness(session, name); + } + public static Optional getSourceTableNameFromSystemTable(SchemaTableName tableName) { return Stream.of(SystemTableHandler.values()) diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadataFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadataFactory.java index 6e30801dfc1e..956e28171980 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadataFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadataFactory.java @@ -57,6 +57,7 @@ public class HiveMetadataFactory private final Executor updateExecutor; private final String trinoVersion; private final HiveRedirectionsProvider hiveRedirectionsProvider; + private final HiveMaterializedViewMetadata hiveMaterializedViewMetadata; private final AccessControlMetadataFactory accessControlMetadataFactory; private final Optional hiveTransactionHeartbeatInterval; private final ScheduledExecutorService heartbeatService; @@ -77,6 +78,7 @@ public HiveMetadataFactory( JsonCodec partitionUpdateCodec, NodeVersion nodeVersion, HiveRedirectionsProvider hiveRedirectionsProvider, + HiveMaterializedViewMetadata hiveMaterializedViewMetadata, AccessControlMetadataFactory accessControlMetadataFactory) { this( @@ -102,6 +104,7 @@ public HiveMetadataFactory( heartbeatService, nodeVersion.toString(), hiveRedirectionsProvider, + hiveMaterializedViewMetadata, accessControlMetadataFactory); } @@ -128,6 +131,7 @@ public HiveMetadataFactory( ScheduledExecutorService heartbeatService, String trinoVersion, HiveRedirectionsProvider hiveRedirectionsProvider, + HiveMaterializedViewMetadata hiveMaterializedViewMetadata, AccessControlMetadataFactory accessControlMetadataFactory) { this.catalogName = requireNonNull(catalogName, "catalogName is null"); @@ -147,6 +151,7 @@ public HiveMetadataFactory( this.partitionUpdateCodec = requireNonNull(partitionUpdateCodec, "partitionUpdateCodec is null"); this.trinoVersion = requireNonNull(trinoVersion, "trinoVersion is null"); this.hiveRedirectionsProvider = requireNonNull(hiveRedirectionsProvider, "hiveRedirectionsProvider is null"); + this.hiveMaterializedViewMetadata = requireNonNull(hiveMaterializedViewMetadata, "hiveMaterializedViewMetadata is null"); this.accessControlMetadataFactory = requireNonNull(accessControlMetadataFactory, "accessControlMetadataFactory is null"); this.hiveTransactionHeartbeatInterval = requireNonNull(hiveTransactionHeartbeatInterval, "hiveTransactionHeartbeatInterval is null"); @@ -191,6 +196,7 @@ public TransactionalMetadata create() trinoVersion, new MetastoreHiveStatisticsProvider(metastore), hiveRedirectionsProvider, + hiveMaterializedViewMetadata, accessControlMetadataFactory.create(metastore)); } } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveModule.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveModule.java index 00c563478574..25b0bd9b6d48 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveModule.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveModule.java @@ -68,8 +68,6 @@ public class HiveModule @Override public void configure(Binder binder) { - binder.install(new HiveHdfsModule()); - binder.bind(DirectoryLister.class).to(CachingDirectoryLister.class).in(Scopes.SINGLETON); configBinder(binder).bindConfig(HiveConfig.class); configBinder(binder).bindConfig(MetastoreConfig.class); @@ -91,6 +89,8 @@ public void configure(Binder binder) binder.bind(LocationService.class).to(HiveLocationService.class).in(Scopes.SINGLETON); newOptionalBinder(binder, HiveRedirectionsProvider.class) .setDefault().to(NoneHiveRedirectionsProvider.class).in(Scopes.SINGLETON); + newOptionalBinder(binder, HiveMaterializedViewMetadata.class) + .setDefault().to(NoneHiveMaterializedViewMetadata.class).in(Scopes.SINGLETON); binder.bind(HiveMetadataFactory.class).in(Scopes.SINGLETON); binder.bind(TransactionalMetadataFactory.class).to(HiveMetadataFactory.class).in(Scopes.SINGLETON); binder.bind(HiveTransactionManager.class).in(Scopes.SINGLETON); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePageSource.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePageSource.java index d08ad11441d1..32738cf1f9c7 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePageSource.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePageSource.java @@ -115,7 +115,9 @@ import static io.trino.spi.type.SmallintType.SMALLINT; import static io.trino.spi.type.TimestampType.TIMESTAMP_MILLIS; import static io.trino.spi.type.TimestampWithTimeZoneType.TIMESTAMP_TZ_MILLIS; +import static io.trino.spi.type.Timestamps.MICROSECONDS_PER_MILLISECOND; import static io.trino.spi.type.TinyintType.TINYINT; +import static java.lang.Math.floorDiv; import static java.lang.String.format; import static java.nio.charset.StandardCharsets.UTF_8; import static java.util.Objects.requireNonNull; @@ -223,7 +225,7 @@ else if (type.equals(TIMESTAMP_MILLIS)) { } else if (type.equals(TIMESTAMP_TZ_MILLIS)) { // used for $file_modified_time - prefilledValue = packDateTimeWithZone(timestampPartitionKey(columnValue, name), DateTimeZone.getDefault().getID()); + prefilledValue = packDateTimeWithZone(floorDiv(timestampPartitionKey(columnValue, name), MICROSECONDS_PER_MILLISECOND), DateTimeZone.getDefault().getID()); } else if (isShortDecimal(type)) { prefilledValue = shortDecimalPartitionKey(columnValue, (DecimalType) type, name); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePartitioningHandle.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePartitioningHandle.java index fc56282c3abf..51195f2c5189 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePartitioningHandle.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePartitioningHandle.java @@ -15,6 +15,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.MoreObjects.ToStringHelper; import io.trino.plugin.hive.util.HiveBucketing.BucketingVersion; import io.trino.spi.connector.ConnectorPartitioningHandle; @@ -82,11 +83,13 @@ public boolean isUsePartitionedBucketing() @Override public String toString() { - return toStringHelper(this) + ToStringHelper helper = toStringHelper(this) .add("buckets", bucketCount) - .add("hiveTypes", hiveTypes) - .add("usePartitionedBucketing", usePartitionedBucketing) - .toString(); + .add("hiveTypes", hiveTypes); + if (usePartitionedBucketing) { + helper.add("usePartitionedBucketing", usePartitionedBucketing); + } + return helper.toString(); } @Override diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveRecordCursor.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveRecordCursor.java index 2f9934a0da07..063d38f7cc60 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveRecordCursor.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveRecordCursor.java @@ -57,7 +57,9 @@ import static io.trino.spi.type.SmallintType.SMALLINT; import static io.trino.spi.type.TimestampType.TIMESTAMP_MILLIS; import static io.trino.spi.type.TimestampWithTimeZoneType.TIMESTAMP_TZ_MILLIS; +import static io.trino.spi.type.Timestamps.MICROSECONDS_PER_MILLISECOND; import static io.trino.spi.type.TinyintType.TINYINT; +import static java.lang.Math.floorDiv; import static java.lang.String.format; import static java.nio.charset.StandardCharsets.UTF_8; import static java.util.Objects.requireNonNull; @@ -147,7 +149,7 @@ else if (TIMESTAMP_MILLIS.equals(type)) { } else if (TIMESTAMP_TZ_MILLIS.equals(type)) { // used for $file_modified_time - longs[columnIndex] = packDateTimeWithZone(timestampPartitionKey(columnValue, name), DateTimeZone.getDefault().getID()); + longs[columnIndex] = packDateTimeWithZone(floorDiv(timestampPartitionKey(columnValue, name), MICROSECONDS_PER_MILLISECOND), DateTimeZone.getDefault().getID()); } else if (isShortDecimal(type)) { longs[columnIndex] = shortDecimalPartitionKey(columnValue, (DecimalType) type, name); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/InternalHiveConnectorFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/InternalHiveConnectorFactory.java index fdea0ec9d561..045a7131c6cd 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/InternalHiveConnectorFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/InternalHiveConnectorFactory.java @@ -92,6 +92,7 @@ public static Connector createConnector(String catalogName, Map new ConnectorObjectNameGeneratorModule(catalogName, "io.trino.plugin.hive", "trino.plugin.hive"), new JsonModule(), new HiveModule(), + new HiveHdfsModule(), new HiveS3Module(), new HiveGcsModule(), new HiveAzureModule(), diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/NoneHiveMaterializedViewMetadata.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/NoneHiveMaterializedViewMetadata.java new file mode 100644 index 000000000000..4c41aad30d47 --- /dev/null +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/NoneHiveMaterializedViewMetadata.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 io.trino.plugin.hive; + +import io.trino.spi.TrinoException; +import io.trino.spi.connector.ConnectorMaterializedViewDefinition; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.MaterializedViewFreshness; +import io.trino.spi.connector.SchemaTableName; + +import java.util.Optional; + +import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; + +public class NoneHiveMaterializedViewMetadata + implements HiveMaterializedViewMetadata +{ + @Override + public void createMaterializedView(ConnectorSession session, SchemaTableName viewName, ConnectorMaterializedViewDefinition definition, boolean replace, boolean ignoreExisting) + { + throw new TrinoException(NOT_SUPPORTED, "This connector does not support creating materialized views"); + } + + @Override + public void dropMaterializedView(ConnectorSession session, SchemaTableName viewName) + { + throw new TrinoException(NOT_SUPPORTED, "This connector does not support dropping materialized views"); + } + + @Override + public Optional getMaterializedView(ConnectorSession session, SchemaTableName viewName) + { + return Optional.empty(); + } + + @Override + public MaterializedViewFreshness getMaterializedViewFreshness(ConnectorSession session, SchemaTableName name) + { + return new MaterializedViewFreshness(false); + } +} diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ViewReaderUtil.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ViewReaderUtil.java index 24b11afea73b..3d0180b5223d 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ViewReaderUtil.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/ViewReaderUtil.java @@ -15,7 +15,7 @@ import com.linkedin.coral.hive.hive2rel.HiveMetastoreClient; import com.linkedin.coral.hive.hive2rel.HiveToRelConverter; -import com.linkedin.coral.presto.rel2presto.RelToPrestoConverter; +import com.linkedin.coral.trino.rel2trino.RelToTrinoConverter; import io.airlift.json.JsonCodec; import io.airlift.json.JsonCodecFactory; import io.airlift.json.ObjectMapperProvider; @@ -163,15 +163,16 @@ public ConnectorViewDefinition decodeViewData(String viewSql, Table table, Catal try { HiveToRelConverter hiveToRelConverter = HiveToRelConverter.create(metastoreClient); RelNode rel = hiveToRelConverter.convertView(table.getDatabaseName(), table.getTableName()); - RelToPrestoConverter rel2Presto = new RelToPrestoConverter(); - String prestoSql = rel2Presto.convert(rel); + RelToTrinoConverter relToTrino = new RelToTrinoConverter(); + String trinoSql = relToTrino.convert(rel); RelDataType rowType = rel.getRowType(); List columns = rowType.getFieldList().stream() .map(field -> new ViewColumn( field.getName(), typeManager.fromSqlType(getTypeString(field.getType())).getTypeId())) .collect(toImmutableList()); - return new ConnectorViewDefinition(prestoSql, + return new ConnectorViewDefinition( + trinoSql, Optional.of(catalogName.toString()), Optional.of(table.getDatabaseName()), columns, diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/SemiTransactionalHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/SemiTransactionalHiveMetastore.java index e8fe14dba0f7..67d6dceb0fef 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/SemiTransactionalHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/SemiTransactionalHiveMetastore.java @@ -2417,8 +2417,8 @@ private static RecursiveDeleteResult recursiveDeleteFiles(HdfsEnvironment hdfsEn private static RecursiveDeleteResult doRecursiveDeleteFiles(FileSystem fileSystem, Path directory, Set queryIds, boolean deleteEmptyDirectories) { - // don't delete hidden presto directories - if (directory.getName().startsWith(".presto")) { + // don't delete hidden Trino directories use by FileHiveMetastore + if (directory.getName().startsWith(".trino")) { return new RecursiveDeleteResult(false, ImmutableList.of()); } @@ -2439,8 +2439,8 @@ private static RecursiveDeleteResult doRecursiveDeleteFiles(FileSystem fileSyste Path filePath = fileStatus.getPath(); String fileName = filePath.getName(); boolean eligible = false; - // never delete presto dot files - if (!fileName.startsWith(".presto")) { + // don't delete hidden Trino directories use by FileHiveMetastore + if (!fileName.startsWith(".trino")) { eligible = queryIds.stream().anyMatch(id -> fileName.startsWith(id) || fileName.endsWith(id)); } if (eligible) { diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/file/FileHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/file/FileHiveMetastore.java index 21db80f79be2..563bd35aff55 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/file/FileHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/file/FileHiveMetastore.java @@ -130,8 +130,8 @@ public class FileHiveMetastore { private static final String PUBLIC_ROLE_NAME = "public"; private static final String ADMIN_ROLE_NAME = "admin"; - private static final String PRESTO_SCHEMA_FILE_NAME = ".prestoSchema"; - private static final String PRESTO_PERMISSIONS_DIRECTORY_NAME = ".prestoPermissions"; + private static final String TRINO_SCHEMA_FILE_NAME = ".trinoSchema"; + private static final String TRINO_PERMISSIONS_DIRECTORY_NAME = ".trinoPermissions"; // todo there should be a way to manage the admins list private static final Set ADMIN_USERS = ImmutableSet.of("admin", "hive", "hdfs"); private static final String ICEBERG_TABLE_TYPE_NAME = "table_type"; @@ -721,7 +721,7 @@ public synchronized void addPartitions(HiveIdentity identity, String databaseNam Partition partition = partitionWithStatistics.getPartition(); verifiedPartition(table, partition); Path partitionMetadataDirectory = getPartitionMetadataDirectory(table, partition.getValues()); - Path schemaPath = new Path(partitionMetadataDirectory, PRESTO_SCHEMA_FILE_NAME); + Path schemaPath = new Path(partitionMetadataDirectory, TRINO_SCHEMA_FILE_NAME); if (metadataFileSystem.exists(schemaPath)) { throw new TrinoException(HIVE_METASTORE_ERROR, "Partition already exists"); } @@ -998,7 +998,7 @@ private synchronized Optional> getAllPartitionNames(HiveIdentity id private boolean isValidPartition(Table table, String partitionName) { try { - return metadataFileSystem.exists(new Path(getPartitionMetadataDirectory(table, partitionName), PRESTO_SCHEMA_FILE_NAME)); + return metadataFileSystem.exists(new Path(getPartitionMetadataDirectory(table, partitionName), TRINO_SCHEMA_FILE_NAME)); } catch (IOException e) { return false; @@ -1218,7 +1218,7 @@ private Path getPartitionMetadataDirectory(Table table, String partitionName) private Path getPermissionsDirectory(Table table) { - return new Path(getTableMetadataDirectory(table), PRESTO_PERMISSIONS_DIRECTORY_NAME); + return new Path(getTableMetadataDirectory(table), TRINO_PERMISSIONS_DIRECTORY_NAME); } private static Path getPermissionsPath(Path permissionsDirectory, HivePrincipal grantee) @@ -1242,7 +1242,7 @@ private List getChildSchemaDirectories(Path metadataDirectory) if (childPath.getName().startsWith(".")) { continue; } - if (metadataFileSystem.isFile(new Path(childPath, PRESTO_SCHEMA_FILE_NAME))) { + if (metadataFileSystem.isFile(new Path(childPath, TRINO_SCHEMA_FILE_NAME))) { childSchemaDirectories.add(childPath); } } @@ -1287,7 +1287,7 @@ private Set readAllPermissions(Path permissionsDirectory) private void deleteMetadataDirectory(Path metadataDirectory) { try { - Path schemaPath = new Path(metadataDirectory, PRESTO_SCHEMA_FILE_NAME); + Path schemaPath = new Path(metadataDirectory, TRINO_SCHEMA_FILE_NAME); if (!metadataFileSystem.isFile(schemaPath)) { // if there is no schema file, assume this is not a database, partition or table return; @@ -1324,7 +1324,7 @@ private void checkVersion(Optional writerVersion) private Optional readSchemaFile(String type, Path metadataDirectory, JsonCodec codec) { - Path schemaPath = new Path(metadataDirectory, PRESTO_SCHEMA_FILE_NAME); + Path schemaPath = new Path(metadataDirectory, TRINO_SCHEMA_FILE_NAME); return readFile(type + " schema", schemaPath, codec); } @@ -1347,7 +1347,7 @@ private Optional readFile(String type, Path path, JsonCodec codec) private void writeSchemaFile(String type, Path directory, JsonCodec codec, T value, boolean overwrite) { - Path schemaPath = new Path(directory, PRESTO_SCHEMA_FILE_NAME); + Path schemaPath = new Path(directory, TRINO_SCHEMA_FILE_NAME); writeFile(type + " schema", schemaPath, codec, value, overwrite); } @@ -1377,7 +1377,7 @@ private void writeFile(String type, Path path, JsonCodec codec, T value, private void deleteSchemaFile(String type, Path metadataDirectory) { try { - if (!metadataFileSystem.delete(new Path(metadataDirectory, PRESTO_SCHEMA_FILE_NAME), false)) { + if (!metadataFileSystem.delete(new Path(metadataDirectory, TRINO_SCHEMA_FILE_NAME), false)) { throw new TrinoException(HIVE_METASTORE_ERROR, "Could not delete " + type + " schema"); } } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OrcFileWriter.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OrcFileWriter.java index ef95c83f7310..53afa9d1eb10 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OrcFileWriter.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OrcFileWriter.java @@ -91,7 +91,6 @@ public OrcFileWriter( ColumnMetadata fileColumnOrcTypes, CompressionKind compression, OrcWriterOptions options, - boolean writeLegacyVersion, int[] fileInputColumnIndexes, Map metadata, Optional> validationInputFactory, @@ -123,7 +122,6 @@ public OrcFileWriter( fileColumnOrcTypes, compression, options, - writeLegacyVersion, metadata, validationInputFactory.isPresent(), validationMode, diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OrcFileWriterFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OrcFileWriterFactory.java index 79ec9163ed8f..43c6d3eb4461 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OrcFileWriterFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OrcFileWriterFactory.java @@ -85,14 +85,12 @@ public class OrcFileWriterFactory private final FileFormatDataSourceStats readStats; private final OrcWriterStats stats = new OrcWriterStats(); private final OrcWriterOptions orcWriterOptions; - private final boolean writeLegacyVersion; @Inject public OrcFileWriterFactory( HdfsEnvironment hdfsEnvironment, TypeManager typeManager, NodeVersion nodeVersion, - OrcWriterConfig orcWriterConfig, FileFormatDataSourceStats readStats, OrcWriterConfig config) { @@ -100,7 +98,6 @@ public OrcFileWriterFactory( hdfsEnvironment, typeManager, nodeVersion, - requireNonNull(orcWriterConfig, "orcWriterConfig is null").isUseLegacyVersion(), readStats, requireNonNull(config, "config is null").toOrcWriterOptions()); } @@ -109,14 +106,12 @@ public OrcFileWriterFactory( HdfsEnvironment hdfsEnvironment, TypeManager typeManager, NodeVersion nodeVersion, - boolean writeLegacyVersion, FileFormatDataSourceStats readStats, OrcWriterOptions orcWriterOptions) { this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); this.typeManager = requireNonNull(typeManager, "typeManager is null"); this.nodeVersion = requireNonNull(nodeVersion, "nodeVersion is null"); - this.writeLegacyVersion = writeLegacyVersion; this.readStats = requireNonNull(readStats, "readStats is null"); this.orcWriterOptions = requireNonNull(orcWriterOptions, "orcWriterOptions is null"); } @@ -215,7 +210,6 @@ public Optional createFileWriter( .withStripeMaxRowCount(getOrcOptimizedWriterMaxStripeRows(session)) .withDictionaryMaxMemory(getOrcOptimizedWriterMaxDictionaryMemory(session)) .withMaxStringStatisticsLimit(getOrcStringStatisticsLimit(session)), - writeLegacyVersion, fileInputColumnIndexes, ImmutableMap.builder() .put(PRESTO_VERSION_NAME, nodeVersion.toString()) diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OrcPageSourceFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OrcPageSourceFactory.java index e69ac8afb82e..f4aa4dc2f58e 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OrcPageSourceFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OrcPageSourceFactory.java @@ -16,6 +16,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; +import io.airlift.slice.Slice; import io.trino.memory.context.AggregatedMemoryContext; import io.trino.orc.NameBasedFieldMapper; import io.trino.orc.OrcColumn; @@ -77,6 +78,8 @@ import static io.trino.orc.OrcReader.INITIAL_BATCH_SIZE; import static io.trino.orc.OrcReader.ProjectedLayout.createProjectedLayout; import static io.trino.orc.OrcReader.ProjectedLayout.fullyProjectedLayout; +import static io.trino.orc.metadata.OrcMetadataWriter.PRESTO_WRITER_ID; +import static io.trino.orc.metadata.OrcMetadataWriter.TRINO_WRITER_ID; import static io.trino.orc.metadata.OrcType.OrcTypeKind.INT; import static io.trino.orc.metadata.OrcType.OrcTypeKind.LONG; import static io.trino.orc.metadata.OrcType.OrcTypeKind.STRUCT; @@ -99,9 +102,11 @@ import static io.trino.plugin.hive.orc.OrcPageSource.ColumnAdaptation.updatedRowColumns; import static io.trino.plugin.hive.orc.OrcPageSource.handleException; import static io.trino.plugin.hive.util.HiveUtil.isDeserializerClass; +import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.spi.type.IntegerType.INTEGER; import static java.lang.String.format; +import static java.nio.charset.StandardCharsets.UTF_8; import static java.util.Locale.ENGLISH; import static java.util.Objects.requireNonNull; import static java.util.function.Function.identity; @@ -258,6 +263,9 @@ private static ConnectorPageSource createOrcPageSource( return new EmptyPageSource(); } OrcReader reader = optionalOrcReader.get(); + if (!originalFile && acidInfo.isPresent() && !acidInfo.get().isOrcAcidVersionValidated()) { + validateOrcAcidVersion(path, reader); + } List fileColumns = reader.getRootColumn().getNestedColumns(); int actualColumnCount = columns.size() + (isFullAcid ? 3 : 0); @@ -298,16 +306,14 @@ private static ConnectorPageSource createOrcPageSource( .collect(Collectors.groupingBy( HiveColumnHandle::getBaseColumnName, mapping( - column -> column.getHiveColumnProjectionInfo().map(HiveColumnProjectionInfo::getDereferenceNames).orElse(ImmutableList.of()), - toList()))); + OrcPageSourceFactory::getDereferencesAsList, toList()))); } else { projectionsByColumnIndex = projections.stream() .collect(Collectors.groupingBy( HiveColumnHandle::getBaseHiveColumnIndex, mapping( - column -> column.getHiveColumnProjectionInfo().map(HiveColumnProjectionInfo::getDereferenceNames).orElse(ImmutableList.of()), - toList()))); + OrcPageSourceFactory::getDereferencesAsList, toList()))); } TupleDomainOrcPredicateBuilder predicateBuilder = TupleDomainOrcPredicate.builder() @@ -439,6 +445,51 @@ else if (transaction.isUpdate()) { } } + private static void validateOrcAcidVersion(Path path, OrcReader reader) + { + // Trino cannot read ORC ACID tables with version < 2 (written by Hive older than 3.0) + // See https://github.com/trinodb/trino/issues/2790#issuecomment-591901728 for more context + + // If we did not manage to validate if ORC ACID version used by table is supported one base don _orc_acid_version metadata file + // we check the data file footer. + + if (reader.getFooter().getNumberOfRows() == 0) { + // file is empty. assuming we are good. We do not want to depend on metadata in such case + // as some hadoop distributions do not write ORC ACID metadata for empty ORC files + return; + } + + int writerId = reader.getFooter().getWriterId().orElseThrow(() -> new TrinoException(HIVE_BAD_DATA, "writerId not set in ORC metadata in " + path)); + if (writerId == TRINO_WRITER_ID || writerId == PRESTO_WRITER_ID) { + // file written by Trino. We are good. + return; + } + + Optional hiveAcidVersion = getHiveAcidVersion(reader); + if (hiveAcidVersion.isEmpty() || hiveAcidVersion.get() < 2) { + throw new TrinoException( + NOT_SUPPORTED, + format("Hive transactional tables are supported since Hive 3.0. Expected `hive.acid.version` in ORC metadata in %s to be >=2 but was %s. " + + "If you have upgraded from an older version of Hive, make sure a major compaction has been run at least once after the upgrade.", + path, + hiveAcidVersion.map(String::valueOf).orElse(""))); + } + } + + private static Optional getHiveAcidVersion(OrcReader reader) + { + Slice slice = reader.getFooter().getUserMetadata().get("hive.acid.version"); + if (slice == null) { + return Optional.empty(); + } + try { + return Optional.of(Integer.valueOf(slice.toString(UTF_8))); + } + catch (RuntimeException ignored) { + return Optional.empty(); + } + } + /** * Recreate the list of fileColumns, updating the names of any whose names have changed in the * corresponding elements of the desiredColumns list. NOTE: this renaming is only applied to @@ -540,4 +591,13 @@ private static OrcColumn getNestedColumn(OrcColumn baseColumn, Optional getDereferencesAsList(HiveColumnHandle column) + { + return column.getHiveColumnProjectionInfo() + .map(info -> info.getDereferenceNames().stream() + .map(dereference -> dereference.toLowerCase(ENGLISH)) + .collect(toImmutableList())) + .orElse(ImmutableList.of()); + } } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OrcWriterConfig.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OrcWriterConfig.java index 4c3a5d2525e6..46c2ebc41dfc 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OrcWriterConfig.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OrcWriterConfig.java @@ -16,9 +16,11 @@ import io.airlift.configuration.Config; import io.airlift.configuration.ConfigDescription; import io.airlift.configuration.DefunctConfig; +import io.airlift.configuration.LegacyConfig; import io.airlift.units.DataSize; import io.trino.orc.OrcWriteValidation.OrcWriteValidationMode; import io.trino.orc.OrcWriterOptions; +import io.trino.orc.OrcWriterOptions.WriterIdentification; import javax.validation.constraints.DecimalMax; import javax.validation.constraints.DecimalMin; @@ -31,7 +33,6 @@ public class OrcWriterConfig private OrcWriterOptions options = new OrcWriterOptions(); private double defaultBloomFilterFpp = 0.05; - private boolean useLegacyVersion; private double validationPercentage; private OrcWriteValidationMode validationMode = OrcWriteValidationMode.BOTH; @@ -137,16 +138,31 @@ public OrcWriterConfig setDefaultBloomFilterFpp(double defaultBloomFilterFpp) return this; } + @Deprecated public boolean isUseLegacyVersion() { - return useLegacyVersion; + return options.getWriterIdentification() == WriterIdentification.LEGACY_HIVE_COMPATIBLE; } - @Config("hive.orc.writer.use-legacy-version-number") + @Deprecated + @LegacyConfig(value = "hive.orc.writer.use-legacy-version-number", replacedBy = "hive.orc.writer.writer-identification") @ConfigDescription("Write ORC files with a version number that is readable by Hive 2.0.0 to 2.2.0") public OrcWriterConfig setUseLegacyVersion(boolean useLegacyVersion) { - this.useLegacyVersion = useLegacyVersion; + this.options = options.withWriterIdentification(useLegacyVersion ? WriterIdentification.LEGACY_HIVE_COMPATIBLE : WriterIdentification.TRINO); + return this; + } + + @NotNull + public WriterIdentification getWriterIdentification() + { + return options.getWriterIdentification(); + } + + @Config("hive.orc.writer.writer-identification") + public OrcWriterConfig setWriterIdentification(WriterIdentification writerIdentification) + { + options = options.withWriterIdentification(writerIdentification); return this; } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/ParquetPageSource.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/ParquetPageSource.java index 8ab0162bd4a1..3f21be4c86c5 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/ParquetPageSource.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/ParquetPageSource.java @@ -14,6 +14,7 @@ package io.trino.plugin.hive.parquet; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Streams; import io.trino.parquet.Field; import io.trino.parquet.ParquetCorruptionException; import io.trino.parquet.reader.ParquetReader; @@ -22,6 +23,7 @@ import io.trino.spi.block.Block; import io.trino.spi.block.LazyBlock; import io.trino.spi.block.LazyBlockLoader; +import io.trino.spi.block.LongArrayBlock; import io.trino.spi.block.RunLengthEncodedBlock; import io.trino.spi.connector.ConnectorPageSource; import io.trino.spi.type.Type; @@ -31,10 +33,12 @@ import java.util.List; import java.util.Optional; +import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; import static io.trino.plugin.hive.HiveErrorCode.HIVE_BAD_DATA; import static io.trino.plugin.hive.HiveErrorCode.HIVE_CURSOR_ERROR; import static java.lang.String.format; +import static java.util.Collections.nCopies; import static java.util.Objects.requireNonNull; public class ParquetPageSource @@ -43,15 +47,53 @@ public class ParquetPageSource private final ParquetReader parquetReader; private final List types; private final List> fields; + /** + * Indicates whether the column at each index should be populated with the + * indices of its rows + */ + private final List rowIndexLocations; private int batchId; private boolean closed; public ParquetPageSource(ParquetReader parquetReader, List types, List> fields) + { + this(parquetReader, types, nCopies(types.size(), false), fields); + } + + /** + * @param types Column types + * @param rowIndexLocations Whether each column should be populated with the indices of its rows + * @param fields List of field descriptions. Empty optionals will result in columns populated with {@code NULL} + */ + public ParquetPageSource( + ParquetReader parquetReader, + List types, + List rowIndexLocations, + List> fields) { this.parquetReader = requireNonNull(parquetReader, "parquetReader is null"); this.types = ImmutableList.copyOf(requireNonNull(types, "types is null")); + this.rowIndexLocations = requireNonNull(rowIndexLocations, "rowIndexLocations is null"); this.fields = ImmutableList.copyOf(requireNonNull(fields, "fields is null")); + + // TODO: Instead of checking that the three list arguments go together correctly, + // we should do something like the ORC reader's ColumnAdatpation, using + // subclasses that contain only the necessary information for each column. + checkArgument( + types.size() == rowIndexLocations.size() && types.size() == fields.size(), + "types, rowIndexLocations, and fields must correspond one-to-one-to-one"); + Streams.forEachPair( + rowIndexLocations.stream(), + fields.stream(), + (isIndexColumn, field) -> checkArgument( + !(isIndexColumn && field.isPresent()), + "Field info for row index column must be empty Optional")); + } + + private boolean isIndexColumn(int column) + { + return rowIndexLocations.get(column); } @Override @@ -91,13 +133,15 @@ public Page getNextPage() } Block[] blocks = new Block[fields.size()]; - for (int fieldId = 0; fieldId < blocks.length; fieldId++) { - Optional field = fields.get(fieldId); - if (field.isPresent()) { - blocks[fieldId] = new LazyBlock(batchSize, new ParquetBlockLoader(field.get())); + for (int column = 0; column < blocks.length; column++) { + if (isIndexColumn(column)) { + blocks[column] = getRowIndexColumn(parquetReader.lastBatchStartRow(), batchSize); } else { - blocks[fieldId] = RunLengthEncodedBlock.create(types.get(fieldId), null, batchSize); + Type type = types.get(column); + blocks[column] = fields.get(column) + .map(field -> new LazyBlock(batchSize, new ParquetBlockLoader(field))) + .orElseGet(() -> RunLengthEncodedBlock.create(type, null, batchSize)); } } return new Page(batchSize, blocks); @@ -145,6 +189,10 @@ public void close() private final class ParquetBlockLoader implements LazyBlockLoader { + /** + * Stores batch ID at instantiation time. Loading fails if the ID + * changes before {@link #load()} is called. + */ private final int expectedBatchId = batchId; private final Field field; private boolean loaded; @@ -158,7 +206,7 @@ public ParquetBlockLoader(Field field) public final Block load() { checkState(!loaded, "Already loaded"); - checkState(batchId == expectedBatchId); + checkState(batchId == expectedBatchId, "Inconsistent state; wrong batch"); Block block; String parquetDataSourceId = parquetReader.getDataSource().getId().toString(); @@ -176,4 +224,13 @@ public final Block load() return block; } } + + private static Block getRowIndexColumn(long baseIndex, int size) + { + long[] rowIndices = new long[size]; + for (int position = 0; position < size; position++) { + rowIndices[position] = baseIndex + position; + } + return new LongArrayBlock(size, Optional.empty(), rowIndices); + } } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/ParquetPageSourceFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/ParquetPageSourceFactory.java index b6c08412cbda..cf312dd95cc9 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/ParquetPageSourceFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/ParquetPageSourceFactory.java @@ -31,6 +31,7 @@ import io.trino.plugin.hive.HiveColumnHandle; import io.trino.plugin.hive.HiveConfig; import io.trino.plugin.hive.HivePageSourceFactory; +import io.trino.plugin.hive.HiveType; import io.trino.plugin.hive.ReaderColumns; import io.trino.plugin.hive.ReaderPageSource; import io.trino.plugin.hive.acid.AcidTransaction; @@ -68,7 +69,6 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Strings.nullToEmpty; -import static com.google.common.collect.ImmutableList.toImmutableList; import static io.trino.memory.context.AggregatedMemoryContext.newSimpleAggregatedMemoryContext; import static io.trino.parquet.ParquetTypeUtils.getColumnIO; import static io.trino.parquet.ParquetTypeUtils.getDescriptors; @@ -87,6 +87,7 @@ import static io.trino.plugin.hive.HiveSessionProperties.isUseParquetColumnNames; import static io.trino.plugin.hive.parquet.ParquetColumnIOConverter.constructField; import static io.trino.plugin.hive.util.HiveUtil.getDeserializerClassName; +import static io.trino.spi.type.BigintType.BIGINT; import static java.lang.String.format; import static java.util.Objects.requireNonNull; import static java.util.stream.Collectors.toUnmodifiableList; @@ -95,6 +96,20 @@ public class ParquetPageSourceFactory implements HivePageSourceFactory { + /** + * If this object is passed as one of the columns for {@code createPageSource}, + * it will be populated as an additional column containing the index of each + * row read. + */ + public static final HiveColumnHandle PARQUET_ROW_INDEX_COLUMN = new HiveColumnHandle( + "$parquet$row_index", + -1, // no real column index + HiveType.HIVE_LONG, + BIGINT, + Optional.empty(), + HiveColumnHandle.ColumnType.SYNTHESIZED, + Optional.empty()); + private static final Set PARQUET_SERDE_CLASS_NAMES = ImmutableSet.builder() .add("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe") .add("parquet.hive.serde.ParquetHiveSerDe") @@ -205,30 +220,30 @@ public static ReaderPageSource createPageSource( requestedSchema = message.orElse(new MessageType(fileSchema.getName(), ImmutableList.of())); messageColumn = getColumnIO(fileSchema, requestedSchema); - ImmutableList.Builder footerBlocks = ImmutableList.builder(); - for (BlockMetaData block : parquetMetadata.getBlocks()) { - long firstDataPage = block.getColumns().get(0).getFirstDataPageOffset(); - if (firstDataPage >= start && firstDataPage < start + length) { - footerBlocks.add(block); - } - } - Map, RichColumnDescriptor> descriptorsByPath = getDescriptors(fileSchema, requestedSchema); TupleDomain parquetTupleDomain = options.isIgnoreStatistics() ? TupleDomain.all() : getParquetTupleDomain(descriptorsByPath, effectivePredicate, fileSchema, useColumnNames); Predicate parquetPredicate = buildPredicate(requestedSchema, parquetTupleDomain, descriptorsByPath, timeZone); + + long nextStart = 0; ImmutableList.Builder blocks = ImmutableList.builder(); - for (BlockMetaData block : footerBlocks.build()) { - if (predicateMatches(parquetPredicate, block, dataSource, descriptorsByPath, parquetTupleDomain)) { + ImmutableList.Builder blockStarts = ImmutableList.builder(); + for (BlockMetaData block : parquetMetadata.getBlocks()) { + long firstDataPage = block.getColumns().get(0).getFirstDataPageOffset(); + if (start <= firstDataPage && firstDataPage < start + length + && predicateMatches(parquetPredicate, block, dataSource, descriptorsByPath, parquetTupleDomain)) { blocks.add(block); + blockStarts.add(nextStart); } + nextStart += block.getRowCount(); } parquetReader = new ParquetReader( Optional.ofNullable(fileMetaData.getCreatedBy()), messageColumn, blocks.build(), + Optional.of(blockStarts.build()), dataSource, timeZone, newSimpleAggregatedMemoryContext(), @@ -267,28 +282,32 @@ public static ReaderPageSource createPageSource( .orElse(columns); for (HiveColumnHandle column : baseColumns) { - checkArgument(column.getColumnType() == REGULAR, "column type must be REGULAR: %s", column); + checkArgument(column == PARQUET_ROW_INDEX_COLUMN || column.getColumnType() == REGULAR, "column type must be REGULAR: %s", column); } - List> parquetFields = baseColumns.stream() - .map(column -> getParquetType(column, fileSchema, useColumnNames)) - .map(Optional::ofNullable) - .collect(toImmutableList()); ImmutableList.Builder trinoTypes = ImmutableList.builder(); ImmutableList.Builder> internalFields = ImmutableList.builder(); - for (int columnIndex = 0; columnIndex < baseColumns.size(); columnIndex++) { - HiveColumnHandle column = baseColumns.get(columnIndex); - Optional parquetField = parquetFields.get(columnIndex); - + ImmutableList.Builder rowIndexColumns = ImmutableList.builder(); + for (HiveColumnHandle column : baseColumns) { trinoTypes.add(column.getBaseType()); - - internalFields.add(parquetField.flatMap(field -> { - String columnName = useColumnNames ? column.getBaseColumnName() : fileSchema.getFields().get(column.getBaseHiveColumnIndex()).getName(); - return constructField(column.getBaseType(), lookupColumnByName(messageColumn, columnName)); - })); + rowIndexColumns.add(column == PARQUET_ROW_INDEX_COLUMN); + if (column == PARQUET_ROW_INDEX_COLUMN) { + internalFields.add(Optional.empty()); + } + else { + internalFields.add(Optional.ofNullable(getParquetType(column, fileSchema, useColumnNames)) + .flatMap(field -> { + String columnName = useColumnNames ? column.getBaseColumnName() : fileSchema.getFields().get(column.getBaseHiveColumnIndex()).getName(); + return constructField(column.getBaseType(), lookupColumnByName(messageColumn, columnName)); + })); + } } - ConnectorPageSource parquetPageSource = new ParquetPageSource(parquetReader, trinoTypes.build(), internalFields.build()); + ConnectorPageSource parquetPageSource = new ParquetPageSource( + parquetReader, + trinoTypes.build(), + rowIndexColumns.build(), + internalFields.build()); return new ReaderPageSource(parquetPageSource, readerProjections); } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/s3/TrinoS3FileSystem.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/s3/TrinoS3FileSystem.java index a577c9ed754f..2b57d09ba69b 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/s3/TrinoS3FileSystem.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/s3/TrinoS3FileSystem.java @@ -344,14 +344,21 @@ public RemoteIterator listFiles(Path path, boolean recursive) { // Either a single level or full listing, depending on the recursive flag, no "directories" // included in either path - return new S3ObjectsV2RemoteIterator(listPrefix(path, OptionalInt.empty(), recursive ? ListingMode.RECURSIVE_FILES_ONLY : ListingMode.SHALLOW_FILES_ONLY)); + return new S3ObjectsV2RemoteIterator(listPath(path, OptionalInt.empty(), recursive ? ListingMode.RECURSIVE_FILES_ONLY : ListingMode.SHALLOW_FILES_ONLY)); + } + + public RemoteIterator listFilesByPrefix(Path prefix, boolean recursive) + { + // Either a single level or full listing, depending on the recursive flag, no "directories" + // included in either path + return new S3ObjectsV2RemoteIterator(listPrefix(keyFromPath(prefix), OptionalInt.empty(), recursive ? ListingMode.RECURSIVE_FILES_ONLY : ListingMode.SHALLOW_FILES_ONLY)); } @Override public RemoteIterator listLocatedStatus(Path path) { STATS.newListLocatedStatusCall(); - return new S3ObjectsV2RemoteIterator(listPrefix(path, OptionalInt.empty(), ListingMode.SHALLOW_ALL)); + return new S3ObjectsV2RemoteIterator(listPath(path, OptionalInt.empty(), ListingMode.SHALLOW_ALL)); } private static final class S3ObjectsV2RemoteIterator @@ -405,7 +412,7 @@ public FileStatus getFileStatus(Path path) if (metadata == null) { // check if this path is a directory - Iterator iterator = listPrefix(path, OptionalInt.of(1), ListingMode.SHALLOW_ALL); + Iterator iterator = listPath(path, OptionalInt.of(1), ListingMode.SHALLOW_ALL); if (iterator.hasNext()) { return new FileStatus(0, true, 1, 0, 0, qualifiedPath(path)); } @@ -605,16 +612,27 @@ public boolean isFilesOnly() } } - private Iterator listPrefix(Path path, OptionalInt initialMaxKeys, ListingMode mode) + /** + * List all objects rooted at the provided path. + */ + private Iterator listPath(Path path, OptionalInt initialMaxKeys, ListingMode mode) { String key = keyFromPath(path); if (!key.isEmpty()) { key += PATH_SEPARATOR; } + return listPrefix(key, initialMaxKeys, mode); + } + + /** + * List all objects whose absolute path matches the provided prefix. + */ + private Iterator listPrefix(String prefix, OptionalInt initialMaxKeys, ListingMode mode) + { ListObjectsV2Request request = new ListObjectsV2Request() .withBucketName(getBucketName(uri)) - .withPrefix(key) + .withPrefix(prefix) .withDelimiter(mode == ListingMode.RECURSIVE_FILES_ONLY ? null : PATH_SEPARATOR) .withMaxKeys(initialMaxKeys.isPresent() ? initialMaxKeys.getAsInt() : null) .withRequesterPays(requesterPaysEnabled); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/IonSqlQueryBuilder.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/s3select/IonSqlQueryBuilder.java similarity index 99% rename from plugin/trino-hive/src/main/java/io/trino/plugin/hive/IonSqlQueryBuilder.java rename to plugin/trino-hive/src/main/java/io/trino/plugin/hive/s3select/IonSqlQueryBuilder.java index 88c071fbc3ec..f19ca5570e9e 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/IonSqlQueryBuilder.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/s3select/IonSqlQueryBuilder.java @@ -11,13 +11,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.hive; +package io.trino.plugin.hive.s3select; import com.google.common.base.Joiner; import com.google.common.collect.ImmutableList; import com.google.common.primitives.Shorts; import com.google.common.primitives.SignedBytes; import io.airlift.slice.Slice; +import io.trino.plugin.hive.HiveColumnHandle; import io.trino.spi.predicate.Domain; import io.trino.spi.predicate.Range; import io.trino.spi.predicate.TupleDomain; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/s3select/S3SelectRecordCursorProvider.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/s3select/S3SelectRecordCursorProvider.java index bf49af4d211c..09bd05549373 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/s3select/S3SelectRecordCursorProvider.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/s3select/S3SelectRecordCursorProvider.java @@ -17,7 +17,6 @@ import io.trino.plugin.hive.HdfsEnvironment; import io.trino.plugin.hive.HiveColumnHandle; import io.trino.plugin.hive.HiveRecordCursorProvider; -import io.trino.plugin.hive.IonSqlQueryBuilder; import io.trino.plugin.hive.ReaderColumns; import io.trino.spi.TrinoException; import io.trino.spi.connector.ConnectorSession; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/HiveUtil.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/HiveUtil.java index ff5e665ca876..fe07bfed8797 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/HiveUtil.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/HiveUtil.java @@ -59,6 +59,7 @@ import org.apache.hadoop.hive.serde2.AbstractSerDe; import org.apache.hadoop.hive.serde2.Deserializer; import org.apache.hadoop.hive.serde2.SerDeException; +import org.apache.hadoop.hive.serde2.avro.AvroSerDe; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo; @@ -434,7 +435,7 @@ private static Class getDeserializerClass(String name) return ParquetHiveSerDe.class; } - if ("org.apache.hadoop.hive.serde2.avro.AvroSerDe".equals(name)) { + if (AvroSerDe.class.getName().equals(name)) { return TrinoAvroSerDe.class; } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/TempFileWriter.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/TempFileWriter.java index 6c4cde4b80a6..7c188dfb6480 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/TempFileWriter.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/TempFileWriter.java @@ -82,7 +82,6 @@ private static OrcWriter createOrcFileWriter(OrcDataSink sink, List types) .withMaxStringStatisticsLimit(DataSize.ofBytes(0)) .withStripeMinSize(DataSize.of(64, MEGABYTE)) .withDictionaryMaxMemory(DataSize.of(1, MEGABYTE)), - false, ImmutableMap.of(), false, OrcWriteValidationMode.BOTH, diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHive.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHive.java index bb2eaa6bc07e..9a5e23b9f83b 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHive.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHive.java @@ -68,6 +68,7 @@ import io.trino.spi.connector.ColumnMetadata; import io.trino.spi.connector.ConnectorBucketNodeMap; import io.trino.spi.connector.ConnectorInsertTableHandle; +import io.trino.spi.connector.ConnectorMaterializedViewDefinition; import io.trino.spi.connector.ConnectorMetadata; import io.trino.spi.connector.ConnectorNewTableLayout; import io.trino.spi.connector.ConnectorNodePartitioningProvider; @@ -120,6 +121,7 @@ import io.trino.spi.type.SqlTimestampWithTimeZone; import io.trino.spi.type.SqlVarbinary; import io.trino.spi.type.Type; +import io.trino.spi.type.TypeId; import io.trino.spi.type.TypeOperators; import io.trino.spi.type.VarcharType; import io.trino.sql.gen.JoinCompiler; @@ -293,6 +295,8 @@ import static java.util.concurrent.Executors.newFixedThreadPool; import static java.util.concurrent.Executors.newScheduledThreadPool; import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static java.util.concurrent.TimeUnit.MINUTES; +import static java.util.concurrent.TimeUnit.SECONDS; import static java.util.stream.Collectors.toList; import static org.apache.hadoop.hive.common.FileUtils.makePartName; import static org.apache.hadoop.hive.metastore.TableType.MANAGED_TABLE; @@ -771,8 +775,8 @@ protected final void setup(String host, int port, String databaseName, String ti hdfsEnvironment, false)), executor, - Duration.valueOf("1m"), - Optional.of(Duration.valueOf("15s")), + new Duration(1, MINUTES), + Optional.of(new Duration(15, SECONDS)), 10000); setup(databaseName, hiveConfig, metastore, hdfsEnvironment); @@ -818,6 +822,25 @@ protected final void setup(String databaseName, HiveConfig hiveConfig, HiveMetas ImmutableMap.of(), TupleDomain.all())); }, + new NoneHiveMaterializedViewMetadata() + { + @Override + public Optional getMaterializedView(ConnectorSession session, SchemaTableName viewName) + { + if (!viewName.getTableName().contains("materialized_view_tester")) { + return Optional.empty(); + } + return Optional.of(new ConnectorMaterializedViewDefinition( + "dummy_view_sql", + Optional.empty(), + Optional.empty(), + Optional.empty(), + ImmutableList.of(new ConnectorMaterializedViewDefinition.Column("abc", TypeId.of("type"))), + Optional.empty(), + "alice", + ImmutableMap.of())); + } + }, SqlStandardAccessControlMetadata::new); transactionManager = new HiveTransactionManager(); splitManager = new HiveSplitManager( @@ -3357,6 +3380,28 @@ public void testApplyRedirection() } } + @Test + public void testMaterializedViewMetadata() + throws Exception + { + SchemaTableName sourceTableName = temporaryTable("materialized_view_tester"); + doCreateEmptyTable(sourceTableName, ORC, CREATE_TABLE_COLUMNS); + SchemaTableName tableName = temporaryTable("mock_table"); + doCreateEmptyTable(tableName, ORC, CREATE_TABLE_COLUMNS); + try (Transaction transaction = newTransaction()) { + ConnectorSession session = newSession(); + ConnectorMetadata metadata = transaction.getMetadata(); + assertThat(metadata.getMaterializedView(session, tableName)).isEmpty(); + Optional result = metadata.getMaterializedView(session, sourceTableName); + assertThat(result).isPresent(); + assertThat(result.get().getOriginalSql()).isEqualTo("dummy_view_sql"); + } + finally { + dropTable(sourceTableName); + dropTable(tableName); + } + } + private ConnectorSession sampleSize(int sampleSize) { return getHiveSession(getHiveConfig() @@ -3881,7 +3926,7 @@ protected Set listAllDataFiles(HdfsContext context, Path path) FileSystem fileSystem = hdfsEnvironment.getFileSystem(context, path); if (fileSystem.exists(path)) { for (FileStatus fileStatus : fileSystem.listStatus(path)) { - if (fileStatus.getPath().getName().startsWith(".presto")) { + if (fileStatus.getPath().getName().startsWith(".trino")) { // skip hidden files } else if (fileStatus.isFile()) { @@ -4976,7 +5021,7 @@ private List listDirectory(HdfsContext context, Path path) return Arrays.stream(fileSystem.listStatus(path)) .map(FileStatus::getPath) .map(Path::getName) - .filter(name -> !name.startsWith(".presto")) + .filter(name -> !name.startsWith(".trino")) .collect(toList()); } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHiveFileSystem.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHiveFileSystem.java index 2dbad7f0b631..35a0c2b45f15 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHiveFileSystem.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHiveFileSystem.java @@ -211,6 +211,7 @@ protected void setup(String host, int port, String databaseName, boolean s3Selec partitionUpdateCodec, new NodeVersion("test_version"), new NoneHiveRedirectionsProvider(), + new NoneHiveMaterializedViewMetadata(), SqlStandardAccessControlMetadata::new); transactionManager = new HiveTransactionManager(); splitManager = new HiveSplitManager( diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveTestUtils.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveTestUtils.java index 89d5bd548568..599e87c57fd4 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveTestUtils.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveTestUtils.java @@ -163,7 +163,6 @@ private static OrcFileWriterFactory getDefaultOrcFileWriterFactory(HdfsEnvironme hdfsEnvironment, TYPE_MANAGER, new NodeVersion("test_version"), - new OrcWriterConfig(), new FileFormatDataSourceStats(), new OrcWriterConfig()); } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestBackgroundHiveSplitLoader.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestBackgroundHiveSplitLoader.java index 95529aaeb421..8b8ca5418c28 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestBackgroundHiveSplitLoader.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestBackgroundHiveSplitLoader.java @@ -111,7 +111,6 @@ import static io.trino.plugin.hive.acid.AcidTransaction.NO_ACID_TRANSACTION; import static io.trino.plugin.hive.util.HiveBucketing.BucketingVersion.BUCKETING_V1; import static io.trino.plugin.hive.util.HiveUtil.getRegularColumnHandles; -import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; import static io.trino.spi.connector.NotPartitionedPartitionHandle.NOT_PARTITIONED; import static io.trino.spi.predicate.TupleDomain.withColumnDomains; import static io.trino.spi.type.IntegerType.INTEGER; @@ -120,8 +119,10 @@ import static java.lang.String.format; import static java.nio.charset.StandardCharsets.UTF_8; import static java.util.concurrent.Executors.newCachedThreadPool; +import static java.util.concurrent.TimeUnit.SECONDS; import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.FILE_INPUT_FORMAT; import static org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_LIB; +import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; @@ -407,7 +408,7 @@ public TupleDomain getCurrentPredicate() return TupleDomain.all(); } }, - Duration.valueOf("1s")); + new Duration(1, SECONDS)); HiveSplitSource hiveSplitSource = hiveSplitSource(backgroundHiveSplitLoader); backgroundHiveSplitLoader.start(hiveSplitSource); @@ -528,7 +529,7 @@ public HivePartitionMetadata next() }, TupleDomain.all(), DynamicFilter.EMPTY, - Duration.valueOf("0s"), + new Duration(0, SECONDS), TYPE_MANAGER, createBucketSplitInfo(Optional.empty(), Optional.empty()), SESSION, @@ -676,7 +677,7 @@ public void testFullAcidTableWithOriginalFiles() } @Test - public void testHive2VersionedFullAcidTableFails() + public void testVersionValidationNoOrcAcidVersionFile() throws Exception { java.nio.file.Path tablePath = Files.createTempDirectory("TestBackgroundHiveSplitLoader"); @@ -687,7 +688,8 @@ public void testHive2VersionedFullAcidTableFails() ImmutableMap.of("transactional", "true")); List filePaths = ImmutableList.of( - tablePath + "/000000_1", // _orc_acid_version does not exist so it's assumed to be "ORC ACID version 0" + tablePath + "/000000_1", + // no /delta_0000002_0000002_0000/_orc_acid_version file tablePath + "/delta_0000002_0000002_0000/bucket_00000"); for (String path : filePaths) { @@ -710,9 +712,103 @@ public void testHive2VersionedFullAcidTableFails() HiveSplitSource hiveSplitSource = hiveSplitSource(backgroundHiveSplitLoader); backgroundHiveSplitLoader.start(hiveSplitSource); - assertThatThrownBy(() -> drain(hiveSplitSource)) - .isInstanceOfSatisfying(TrinoException.class, e -> assertEquals(NOT_SUPPORTED.toErrorCode(), e.getErrorCode())) - .hasMessage("Hive transactional tables are supported with Hive 3.0 and only after a major compaction has been run"); + + // We should have it marked in all splits that further ORC ACID validation is required + assertThat(drainSplits(hiveSplitSource)).extracting(HiveSplit::getAcidInfo) + .allMatch(Optional::isPresent) + .extracting(Optional::get) + .noneMatch(AcidInfo::isOrcAcidVersionValidated); + + deleteRecursively(tablePath, ALLOW_INSECURE); + } + + @Test + public void testVersionValidationOrcAcidVersionFileHasVersion2() + throws Exception + { + java.nio.file.Path tablePath = Files.createTempDirectory("TestBackgroundHiveSplitLoader"); + Table table = table( + tablePath.toString(), + ImmutableList.of(), + Optional.empty(), + ImmutableMap.of("transactional", "true")); + + List filePaths = ImmutableList.of( + tablePath + "/000000_1", // _orc_acid_version does not exist so it's assumed to be "ORC ACID version 0" + tablePath + "/delta_0000002_0000002_0000/_orc_acid_version", + tablePath + "/delta_0000002_0000002_0000/bucket_00000"); + + for (String path : filePaths) { + File file = new File(path); + assertTrue(file.getParentFile().exists() || file.getParentFile().mkdirs(), "Failed creating directory " + file.getParentFile()); + createOrcAcidFile(file, 2); + } + + // ValidWriteIdsList is of format $.
:::: + // This writeId list has high watermark transaction=3 + ValidReaderWriteIdList validWriteIdsList = new ValidReaderWriteIdList(format("4$%s.%s:3:9223372036854775807::", table.getDatabaseName(), table.getTableName())); + + BackgroundHiveSplitLoader backgroundHiveSplitLoader = backgroundHiveSplitLoader( + HDFS_ENVIRONMENT, + TupleDomain.all(), + Optional.empty(), + table, + Optional.empty(), + Optional.of(validWriteIdsList)); + + HiveSplitSource hiveSplitSource = hiveSplitSource(backgroundHiveSplitLoader); + backgroundHiveSplitLoader.start(hiveSplitSource); + + // We should have it marked in all splits that NO further ORC ACID validation is required + assertThat(drainSplits(hiveSplitSource)).extracting(HiveSplit::getAcidInfo) + .allMatch(acidInfo -> acidInfo.isEmpty() || acidInfo.get().isOrcAcidVersionValidated()); + + deleteRecursively(tablePath, ALLOW_INSECURE); + } + + @Test + public void testVersionValidationOrcAcidVersionFileHasVersion1() + throws Exception + { + java.nio.file.Path tablePath = Files.createTempDirectory("TestBackgroundHiveSplitLoader"); + Table table = table( + tablePath.toString(), + ImmutableList.of(), + Optional.empty(), + ImmutableMap.of("transactional", "true")); + + List filePaths = ImmutableList.of( + tablePath + "/000000_1", + tablePath + "/delta_0000002_0000002_0000/_orc_acid_version", + tablePath + "/delta_0000002_0000002_0000/bucket_00000"); + + for (String path : filePaths) { + File file = new File(path); + assertTrue(file.getParentFile().exists() || file.getParentFile().mkdirs(), "Failed creating directory " + file.getParentFile()); + // _orc_acid_version_exists but has version 1 + createOrcAcidFile(file, 1); + } + + // ValidWriteIdsList is of format $.
:::: + // This writeId list has high watermark transaction=3 + ValidReaderWriteIdList validWriteIdsList = new ValidReaderWriteIdList(format("4$%s.%s:3:9223372036854775807::", table.getDatabaseName(), table.getTableName())); + + BackgroundHiveSplitLoader backgroundHiveSplitLoader = backgroundHiveSplitLoader( + HDFS_ENVIRONMENT, + TupleDomain.all(), + Optional.empty(), + table, + Optional.empty(), + Optional.of(validWriteIdsList)); + + HiveSplitSource hiveSplitSource = hiveSplitSource(backgroundHiveSplitLoader); + backgroundHiveSplitLoader.start(hiveSplitSource); + + // We should have it marked in all splits that further ORC ACID validation is required + assertThat(drainSplits(hiveSplitSource)).extracting(HiveSplit::getAcidInfo) + .allMatch(Optional::isPresent) + .extracting(Optional::get) + .noneMatch(AcidInfo::isOrcAcidVersionValidated); deleteRecursively(tablePath, ALLOW_INSECURE); } @@ -821,9 +917,15 @@ public void testBuildManifestFileIteratorNestedDirectory() private static void createOrcAcidFile(File file) throws IOException + { + createOrcAcidFile(file, 2); + } + + private static void createOrcAcidFile(File file, int orcAcidVersion) + throws IOException { if (file.getName().equals("_orc_acid_version")) { - Files.write(file.toPath(), "2".getBytes(UTF_8)); + Files.write(file.toPath(), String.valueOf(orcAcidVersion).getBytes(UTF_8)); return; } checkState(file.createNewFile(), "Failed to create file %s", file); @@ -922,7 +1024,7 @@ private BackgroundHiveSplitLoader backgroundHiveSplitLoader( hdfsEnvironment, compactEffectivePredicate, DynamicFilter.EMPTY, - Duration.valueOf("0s"), + new Duration(0, SECONDS), hiveBucketFilter, table, bucketHandle, @@ -984,7 +1086,7 @@ private BackgroundHiveSplitLoader backgroundHiveSplitLoader(List fileModifiedTimeMap = new HashMap<>(); for (int i = 0; i < results.getRowCount(); i++) { MaterializedRow row = results.getMaterializedRows().get(i); @@ -4275,7 +4279,7 @@ public void testFileModifiedTimeHiddenColumn() int col1 = (int) row.getField(1); Instant fileModifiedTime = ((ZonedDateTime) row.getField(2)).toInstant(); - assertTrue(fileModifiedTime.toEpochMilli() > (testStartTime - 2_000)); + assertThat(fileModifiedTime.toEpochMilli()).isCloseTo(testStartTime, offset(2000L)); assertEquals(col0 % 3, col1); if (fileModifiedTimeMap.containsKey(col1)) { assertEquals(fileModifiedTimeMap.get(col1), fileModifiedTime); diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestOrcWriterConfig.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestOrcWriterConfig.java index b3e3f3c82769..c9c533c5350c 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestOrcWriterConfig.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestOrcWriterConfig.java @@ -16,6 +16,7 @@ import com.google.common.collect.ImmutableMap; import io.airlift.units.DataSize; import io.trino.orc.OrcWriteValidation.OrcWriteValidationMode; +import io.trino.orc.OrcWriterOptions.WriterIdentification; import io.trino.plugin.hive.orc.OrcWriterConfig; import org.testng.annotations.Test; @@ -41,7 +42,7 @@ public void testDefaults() .setStringStatisticsLimit(DataSize.ofBytes(64)) .setMaxCompressionBufferSize(DataSize.of(256, KILOBYTE)) .setDefaultBloomFilterFpp(0.05) - .setUseLegacyVersion(false) + .setWriterIdentification(WriterIdentification.TRINO) .setValidationPercentage(0.0) .setValidationMode(OrcWriteValidationMode.BOTH)); } @@ -58,7 +59,7 @@ public void testExplicitPropertyMappings() .put("hive.orc.writer.string-statistics-limit", "17MB") .put("hive.orc.writer.max-compression-buffer-size", "19MB") .put("hive.orc.default-bloom-filter-fpp", "0.96") - .put("hive.orc.writer.use-legacy-version-number", "true") + .put("hive.orc.writer.writer-identification", "LEGACY_HIVE_COMPATIBLE") .put("hive.orc.writer.validation-percentage", "0.16") .put("hive.orc.writer.validation-mode", "DETAILED") .build(); @@ -72,7 +73,7 @@ public void testExplicitPropertyMappings() .setStringStatisticsLimit(DataSize.of(17, MEGABYTE)) .setMaxCompressionBufferSize(DataSize.of(19, MEGABYTE)) .setDefaultBloomFilterFpp(0.96) - .setUseLegacyVersion(true) + .setWriterIdentification(WriterIdentification.LEGACY_HIVE_COMPATIBLE) .setValidationPercentage(0.16) .setValidationMode(OrcWriteValidationMode.DETAILED); diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/benchmark/FileFormat.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/benchmark/FileFormat.java index 02623edc5ad7..de353b8f993a 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/benchmark/FileFormat.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/benchmark/FileFormat.java @@ -603,7 +603,6 @@ public PrestoOrcFormatWriter(File targetFile, List columnNames, List false); + assertRead(ImmutableMap.of(NATION_KEY, 0, NAME, 1, REGION_KEY, 2, COMMENT, 3), OptionalLong.empty(), Optional.empty(), nationKey -> false); } @Test @@ -122,6 +123,36 @@ public void testDeletedRows() assertRead(ALL_COLUMNS, OptionalLong.empty(), acidInfo, nationKey -> nationKey == 5 || nationKey == 19); } + @Test + public void testReadWithAcidVersionValidationHive3() + { + File tableFile = new File(TestOrcPageSourceFactory.class.getClassLoader().getResource("acid_version_validation/acid_version_hive_3/00000_0").getPath()); + String tablePath = tableFile.getParent(); + + Optional acidInfo = AcidInfo.builder(new Path(tablePath)) + .setOrcAcidVersionValidated(false) + .build(); + + List result = readFile(Map.of(), OptionalLong.empty(), acidInfo, tableFile.getPath(), 625); + assertEquals(result.size(), 1); + } + + @Test + public void testReadWithAcidVersionValidationNoVersionInMetadata() + { + File tableFile = new File(TestOrcPageSourceFactory.class.getClassLoader().getResource("acid_version_validation/no_orc_acid_version_in_metadata/00000_0").getPath()); + String tablePath = tableFile.getParent(); + + Optional acidInfo = AcidInfo.builder(new Path(tablePath)) + .setOrcAcidVersionValidated(false) + .build(); + + Assertions.assertThatThrownBy(() -> readFile(Map.of(), OptionalLong.empty(), acidInfo, tableFile.getPath(), 730)) + .hasMessageMatching("Hive transactional tables are supported since Hive 3.0. Expected `hive.acid.version` in ORC metadata" + + " in .*/acid_version_validation/no_orc_acid_version_in_metadata/00000_0 to be >=2 but was ." + + " If you have upgraded from an older version of Hive, make sure a major compaction has been run at least once after the upgrade."); + } + @Test public void testFullFileReadOriginalFilesTable() { @@ -131,6 +162,7 @@ public void testFullFileReadOriginalFilesTable() AcidInfo acidInfo = AcidInfo.builder(new Path(tablePath)) .addDeleteDelta(new Path(tablePath, deleteDeltaSubdir(10000001, 10000001, 0))) .addOriginalFile(new Path(tablePath, "000000_0"), 1780, 0) + .setOrcAcidVersionValidated(true) .buildWithRequiredOriginalFiles(0); List expected = expectedResult(OptionalLong.empty(), nationKey -> nationKey == 24, 1); diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/orc/TestOrcPredicates.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/orc/TestOrcPredicates.java index 681487000d88..596fd4a8aef3 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/orc/TestOrcPredicates.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/orc/TestOrcPredicates.java @@ -98,7 +98,7 @@ private void testOrcPredicates(ConnectorSession session) file.delete(); try { // Write data - OrcFileWriterFactory writerFactory = new OrcFileWriterFactory(HDFS_ENVIRONMENT, TYPE_MANAGER, new NodeVersion("test"), false, STATS, new OrcWriterOptions()); + OrcFileWriterFactory writerFactory = new OrcFileWriterFactory(HDFS_ENVIRONMENT, TYPE_MANAGER, new NodeVersion("test"), STATS, new OrcWriterOptions()); FileSplit split = createTestFileTrino(file.getAbsolutePath(), ORC, HiveCompressionCodec.NONE, columnsToWrite, session, NUM_ROWS, writerFactory); TupleDomain testingPredicate; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/rubix/TestRubixConfig.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/rubix/TestRubixConfig.java index f2b06be96c0b..460986cc218f 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/rubix/TestRubixConfig.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/rubix/TestRubixConfig.java @@ -28,6 +28,8 @@ import static io.airlift.configuration.testing.ConfigAssertions.assertRecordedDefaults; import static io.airlift.configuration.testing.ConfigAssertions.recordDefaults; import static io.airlift.testing.ValidationAssertions.assertFailsValidation; +import static java.util.concurrent.TimeUnit.DAYS; +import static java.util.concurrent.TimeUnit.HOURS; import static org.assertj.core.api.Assertions.assertThat; public class TestRubixConfig @@ -39,7 +41,7 @@ public void testDefaults() .setBookKeeperServerPort(CacheConfig.DEFAULT_BOOKKEEPER_SERVER_PORT) .setDataTransferServerPort(CacheConfig.DEFAULT_DATA_TRANSFER_SERVER_PORT) .setCacheLocation(null) - .setCacheTtl(Duration.valueOf("7d")) + .setCacheTtl(new Duration(7, DAYS)) .setDiskUsagePercentage(CacheConfig.DEFAULT_DATA_CACHE_FULLNESS) .setReadMode(RubixConfig.ReadMode.ASYNC) .setStartServerOnCoordinator(false)); @@ -61,7 +63,7 @@ public void testExplicitPropertyMappings() RubixConfig expected = new RubixConfig() .setReadMode(RubixConfig.ReadMode.READ_THROUGH) .setCacheLocation("/some-directory") - .setCacheTtl(Duration.valueOf("5h")) + .setCacheTtl(new Duration(5, HOURS)) .setDiskUsagePercentage(90) .setBookKeeperServerPort(1234) .setDataTransferServerPort(1235) diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/TestS3SecurityMappingConfig.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/TestS3SecurityMappingConfig.java index 6f8bc4b36534..304a55fcf043 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/TestS3SecurityMappingConfig.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/TestS3SecurityMappingConfig.java @@ -25,6 +25,7 @@ import static io.airlift.configuration.testing.ConfigAssertions.assertFullMapping; import static io.airlift.configuration.testing.ConfigAssertions.assertRecordedDefaults; import static io.airlift.configuration.testing.ConfigAssertions.recordDefaults; +import static java.util.concurrent.TimeUnit.SECONDS; public class TestS3SecurityMappingConfig { @@ -57,7 +58,7 @@ public void testExplicitPropertyMappings() .setConfigFile(securityMappingConfigFile.toFile()) .setRoleCredentialName("iam-role-credential-name") .setKmsKeyIdCredentialName("kms-key-id-credential-name") - .setRefreshPeriod(Duration.valueOf("1s")) + .setRefreshPeriod(new Duration(1, SECONDS)) .setColonReplacement("#"); assertFullMapping(properties, expected); diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestIonSqlQueryBuilder.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3select/TestIonSqlQueryBuilder.java similarity index 98% rename from plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestIonSqlQueryBuilder.java rename to plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3select/TestIonSqlQueryBuilder.java index 65042fb2219e..89cd241120a2 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestIonSqlQueryBuilder.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3select/TestIonSqlQueryBuilder.java @@ -11,10 +11,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.hive; +package io.trino.plugin.hive.s3select; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import io.trino.plugin.hive.HiveColumnHandle; +import io.trino.plugin.hive.HiveType; import io.trino.spi.predicate.Domain; import io.trino.spi.predicate.Range; import io.trino.spi.predicate.SortedRangeSet; diff --git a/plugin/trino-hive/src/test/resources/acid_version_validation/acid_version_hive_3/00000_0 b/plugin/trino-hive/src/test/resources/acid_version_validation/acid_version_hive_3/00000_0 new file mode 100644 index 000000000000..4793ca4ada30 Binary files /dev/null and b/plugin/trino-hive/src/test/resources/acid_version_validation/acid_version_hive_3/00000_0 differ diff --git a/plugin/trino-hive/src/test/resources/acid_version_validation/no_orc_acid_version_in_metadata/00000_0 b/plugin/trino-hive/src/test/resources/acid_version_validation/no_orc_acid_version_in_metadata/00000_0 new file mode 100644 index 000000000000..bc19ae90c9a5 Binary files /dev/null and b/plugin/trino-hive/src/test/resources/acid_version_validation/no_orc_acid_version_in_metadata/00000_0 differ diff --git a/plugin/trino-iceberg/pom.xml b/plugin/trino-iceberg/pom.xml index 5587307e6a2e..1f8513591b6e 100644 --- a/plugin/trino-iceberg/pom.xml +++ b/plugin/trino-iceberg/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergFileWriterFactory.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergFileWriterFactory.java index 1a5aecf77e50..d3c07dcda263 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergFileWriterFactory.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergFileWriterFactory.java @@ -220,7 +220,6 @@ private IcebergFileWriter createOrcWriter( .withStripeMaxRowCount(getOrcWriterMaxStripeRows(session)) .withDictionaryMaxMemory(getOrcWriterMaxDictionaryMemory(session)) .withMaxStringStatisticsLimit(getOrcStringStatisticsLimit(session)), - false, IntStream.range(0, fileColumnNames.size()).toArray(), ImmutableMap.builder() .put(PRESTO_VERSION_NAME, nodeVersion.toString()) diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java index 8e39ff0743b8..2befb6cec9b7 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java @@ -976,28 +976,19 @@ private boolean isMaterializedView(Table table) && table.getParameters().containsKey(STORAGE_TABLE); } - private boolean isMaterializedView(ConnectorSession session, SchemaTableName schemaTableName) - { - final HiveIdentity identity = new HiveIdentity(session); - if (metastore.getTable(identity, schemaTableName.getSchemaName(), schemaTableName.getTableName()).isPresent()) { - Table table = metastore.getTable(identity, schemaTableName.getSchemaName(), schemaTableName.getTableName()).get(); - return isMaterializedView(table); - } - return false; - } - @Override public Optional getMaterializedView(ConnectorSession session, SchemaTableName viewName) { - Optional
materializedViewOptional = metastore.getTable(new HiveIdentity(session), viewName.getSchemaName(), viewName.getTableName()); - if (materializedViewOptional.isEmpty()) { + Optional
tableOptional = metastore.getTable(new HiveIdentity(session), viewName.getSchemaName(), viewName.getTableName()); + if (tableOptional.isEmpty()) { return Optional.empty(); } - if (!isMaterializedView(session, viewName)) { + + if (!isMaterializedView(tableOptional.get())) { return Optional.empty(); } - Table materializedView = materializedViewOptional.get(); + Table materializedView = tableOptional.get(); ConnectorMaterializedViewDefinition definition = decodeMaterializedViewData(materializedView.getViewOriginalText() .orElseThrow(() -> new TrinoException(HIVE_INVALID_METADATA, "No view original text: " + viewName))); @@ -1010,7 +1001,7 @@ public Optional getMaterializedView(Connect Optional.of(viewName.getSchemaName()), definition.getColumns(), definition.getComment(), - Optional.of(materializedView.getOwner()), + materializedView.getOwner(), new HashMap<>(materializedView.getParameters()))); } diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergModule.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergModule.java index 4f4b85faa40d..7e86ab081818 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergModule.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergModule.java @@ -19,7 +19,6 @@ import com.google.inject.multibindings.Multibinder; import io.trino.plugin.hive.FileFormatDataSourceStats; import io.trino.plugin.hive.HiveConfig; -import io.trino.plugin.hive.HiveHdfsModule; import io.trino.plugin.hive.HiveNodePartitioningProvider; import io.trino.plugin.hive.metastore.MetastoreConfig; import io.trino.plugin.hive.orc.OrcReaderConfig; @@ -43,8 +42,6 @@ public class IcebergModule @Override public void configure(Binder binder) { - binder.install(new HiveHdfsModule()); - binder.bind(IcebergTransactionManager.class).in(Scopes.SINGLETON); configBinder(binder).bindConfig(HiveConfig.class); diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergOrcFileWriter.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergOrcFileWriter.java index cbe3aa1b0a2d..d0d2c60eea2b 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergOrcFileWriter.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergOrcFileWriter.java @@ -72,14 +72,13 @@ public IcebergOrcFileWriter( ColumnMetadata fileColumnOrcTypes, CompressionKind compression, OrcWriterOptions options, - boolean writeLegacyVersion, int[] fileInputColumnIndexes, Map metadata, Optional> validationInputFactory, OrcWriteValidation.OrcWriteValidationMode validationMode, OrcWriterStats stats) { - super(orcDataSink, WriterKind.INSERT, NO_ACID_TRANSACTION, false, OptionalInt.empty(), rollbackAction, columnNames, fileColumnTypes, fileColumnOrcTypes, compression, options, writeLegacyVersion, fileInputColumnIndexes, metadata, validationInputFactory, validationMode, stats); + super(orcDataSink, WriterKind.INSERT, NO_ACID_TRANSACTION, false, OptionalInt.empty(), rollbackAction, columnNames, fileColumnTypes, fileColumnOrcTypes, compression, options, fileInputColumnIndexes, metadata, validationInputFactory, validationMode, stats); this.icebergSchema = requireNonNull(icebergSchema, "icebergSchema is null"); orcColumns = fileColumnOrcTypes; } diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProvider.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProvider.java index 320d97bd260e..992761d2a847 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProvider.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProvider.java @@ -471,7 +471,7 @@ private static ConnectorPageSource createParquetPageSource( List blocks = new ArrayList<>(); for (BlockMetaData block : parquetMetadata.getBlocks()) { long firstDataPage = block.getColumns().get(0).getFirstDataPageOffset(); - if ((firstDataPage >= start) && (firstDataPage < (start + length)) && + if (start <= firstDataPage && firstDataPage < start + length && predicateMatches(parquetPredicate, block, dataSource, descriptorsByPath, parquetTupleDomain)) { blocks.add(block); } @@ -482,6 +482,7 @@ private static ConnectorPageSource createParquetPageSource( Optional.ofNullable(fileMetaData.getCreatedBy()), messageColumnIO, blocks, + Optional.empty(), dataSource, UTC, systemMemoryContext, diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/InternalIcebergConnectorFactory.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/InternalIcebergConnectorFactory.java index 27627cf09c70..178cc7481e53 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/InternalIcebergConnectorFactory.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/InternalIcebergConnectorFactory.java @@ -29,6 +29,7 @@ import io.trino.plugin.base.jmx.ConnectorObjectNameGeneratorModule; import io.trino.plugin.base.jmx.MBeanServerModule; import io.trino.plugin.base.security.AllowAllAccessControl; +import io.trino.plugin.hive.HiveHdfsModule; import io.trino.plugin.hive.NodeVersion; import io.trino.plugin.hive.authentication.HiveAuthenticationModule; import io.trino.plugin.hive.azure.HiveAzureModule; @@ -68,6 +69,7 @@ public static Connector createConnector(String catalogName, Map new JsonModule(), new IcebergModule(), new IcebergMetastoreModule(), + new HiveHdfsModule(), new HiveS3Module(), new HiveGcsModule(), new HiveAzureModule(), diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestMetadataQueryOptimization.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestMetadataQueryOptimization.java index 261a6b2ba9ea..eca2f5e0451b 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestMetadataQueryOptimization.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestMetadataQueryOptimization.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.prestosql.plugin.iceberg; +package io.trino.plugin.iceberg; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -20,7 +20,6 @@ import io.trino.plugin.hive.authentication.HiveIdentity; import io.trino.plugin.hive.metastore.Database; import io.trino.plugin.hive.metastore.HiveMetastore; -import io.trino.plugin.iceberg.TestingIcebergConnectorFactory; import io.trino.spi.security.PrincipalType; import io.trino.sql.planner.assertions.BasePushdownPlanTest; import io.trino.sql.tree.LongLiteral; diff --git a/plugin/trino-jmx/pom.xml b/plugin/trino-jmx/pom.xml index b8728b143448..2a15879d17bc 100644 --- a/plugin/trino-jmx/pom.xml +++ b/plugin/trino-jmx/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-kafka/pom.xml b/plugin/trino-kafka/pom.xml index 61774692543e..98fa0644af25 100644 --- a/plugin/trino-kafka/pom.xml +++ b/plugin/trino-kafka/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml @@ -84,6 +84,11 @@ kafka-schema-registry-client + + javax.annotation + javax.annotation-api + + javax.inject javax.inject diff --git a/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/KafkaClientsModule.java b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/KafkaClientsModule.java new file mode 100644 index 000000000000..95c2f614703e --- /dev/null +++ b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/KafkaClientsModule.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 io.trino.plugin.kafka; + +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Scopes; +import io.airlift.configuration.AbstractConfigurationAwareModule; +import io.trino.plugin.kafka.security.ForKafkaSsl; +import org.apache.kafka.common.security.auth.SecurityProtocol; + +import static io.airlift.configuration.ConditionalModule.installModuleIf; +import static io.airlift.configuration.ConfigBinder.configBinder; + +public class KafkaClientsModule + extends AbstractConfigurationAwareModule +{ + @Override + protected void setup(Binder binder) + { + configBinder(binder).bindConfig(KafkaSecurityConfig.class); + installClientModule(SecurityProtocol.PLAINTEXT, KafkaClientsModule::configurePlainText); + installClientModule(SecurityProtocol.SSL, KafkaClientsModule::configureSsl); + } + + private void installClientModule(SecurityProtocol securityProtocol, Module module) + { + install(installModuleIf( + KafkaSecurityConfig.class, + config -> config.getSecurityProtocol().equals(securityProtocol), + module)); + } + + private static void configurePlainText(Binder binder) + { + binder.bind(KafkaConsumerFactory.class).to(PlainTextKafkaConsumerFactory.class).in(Scopes.SINGLETON); + binder.bind(KafkaProducerFactory.class).to(PlainTextKafkaProducerFactory.class).in(Scopes.SINGLETON); + binder.bind(KafkaAdminFactory.class).to(PlainTextKafkaAdminFactory.class).in(Scopes.SINGLETON); + } + + private static void configureSsl(Binder binder) + { + binder.bind(KafkaConsumerFactory.class).annotatedWith(ForKafkaSsl.class).to(PlainTextKafkaConsumerFactory.class).in(Scopes.SINGLETON); + binder.bind(KafkaProducerFactory.class).annotatedWith(ForKafkaSsl.class).to(PlainTextKafkaProducerFactory.class).in(Scopes.SINGLETON); + binder.bind(KafkaAdminFactory.class).annotatedWith(ForKafkaSsl.class).to(PlainTextKafkaAdminFactory.class).in(Scopes.SINGLETON); + + binder.bind(KafkaConsumerFactory.class).to(SslKafkaConsumerFactory.class).in(Scopes.SINGLETON); + binder.bind(KafkaProducerFactory.class).to(SslKafkaProducerFactory.class).in(Scopes.SINGLETON); + binder.bind(KafkaAdminFactory.class).to(SslKafkaAdminFactory.class).in(Scopes.SINGLETON); + } +} diff --git a/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/KafkaPlugin.java b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/KafkaPlugin.java index c7487c33c020..1eeaa143c770 100644 --- a/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/KafkaPlugin.java +++ b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/KafkaPlugin.java @@ -14,7 +14,10 @@ package io.trino.plugin.kafka; import com.google.common.collect.ImmutableList; +import com.google.inject.Binder; import com.google.inject.Module; +import io.airlift.configuration.AbstractConfigurationAwareModule; +import io.trino.plugin.kafka.security.KafkaSecurityModule; import io.trino.spi.Plugin; import io.trino.spi.connector.ConnectorFactory; @@ -23,10 +26,14 @@ public class KafkaPlugin implements Plugin { - public static final Module DEFAULT_EXTENSION = binder -> { - binder.install(new KafkaConsumerModule()); - binder.install(new KafkaProducerModule()); - binder.install(new KafkaAdminModule()); + public static final Module DEFAULT_EXTENSION = new AbstractConfigurationAwareModule() + { + @Override + protected void setup(Binder binder) + { + install(new KafkaClientsModule()); + install(new KafkaSecurityModule()); + } }; private final Module extension; diff --git a/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/KafkaProducerFactory.java b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/KafkaProducerFactory.java index ad049a125845..ee0287dedffe 100644 --- a/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/KafkaProducerFactory.java +++ b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/KafkaProducerFactory.java @@ -11,7 +11,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package io.trino.plugin.kafka; import io.trino.spi.connector.ConnectorSession; diff --git a/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/KafkaSecurityConfig.java b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/KafkaSecurityConfig.java new file mode 100644 index 000000000000..e806cc97002b --- /dev/null +++ b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/KafkaSecurityConfig.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 io.trino.plugin.kafka; + +import io.airlift.configuration.Config; +import io.airlift.configuration.ConfigDescription; +import org.apache.kafka.common.security.auth.SecurityProtocol; + +import javax.annotation.PostConstruct; +import javax.validation.constraints.NotNull; + +import static com.google.common.base.Preconditions.checkState; +import static java.lang.String.format; +import static org.apache.kafka.common.security.auth.SecurityProtocol.PLAINTEXT; +import static org.apache.kafka.common.security.auth.SecurityProtocol.SSL; + +public class KafkaSecurityConfig +{ + private SecurityProtocol securityProtocol = PLAINTEXT; + + @NotNull + public SecurityProtocol getSecurityProtocol() + { + return securityProtocol; + } + + @Config("kafka.security-protocol") + @ConfigDescription("Kafka communication security protocol") + public KafkaSecurityConfig setSecurityProtocol(SecurityProtocol securityProtocol) + { + this.securityProtocol = securityProtocol; + return this; + } + + @PostConstruct + public void validate() + { + checkState( + securityProtocol.equals(PLAINTEXT) || securityProtocol.equals(SSL), + format("Only %s and %s security protocols are supported", PLAINTEXT, SSL)); + } +} diff --git a/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/PlainTextKafkaAdminFactory.java b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/PlainTextKafkaAdminFactory.java index d775e7130a48..0b710f31929e 100644 --- a/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/PlainTextKafkaAdminFactory.java +++ b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/PlainTextKafkaAdminFactory.java @@ -16,6 +16,7 @@ import io.trino.spi.HostAddress; import io.trino.spi.connector.ConnectorSession; +import org.apache.kafka.common.security.auth.SecurityProtocol; import javax.inject.Inject; @@ -24,19 +25,25 @@ import static java.util.Objects.requireNonNull; import static java.util.stream.Collectors.joining; +import static org.apache.kafka.clients.CommonClientConfigs.SECURITY_PROTOCOL_CONFIG; import static org.apache.kafka.clients.consumer.ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG; public class PlainTextKafkaAdminFactory implements KafkaAdminFactory { private final Set nodes; + private final SecurityProtocol securityProtocol; @Inject - public PlainTextKafkaAdminFactory(KafkaConfig kafkaConfig) + public PlainTextKafkaAdminFactory( + KafkaConfig kafkaConfig, + KafkaSecurityConfig securityConfig) { requireNonNull(kafkaConfig, "kafkaConfig is null"); + requireNonNull(securityConfig, "securityConfig is null"); nodes = kafkaConfig.getNodes(); + securityProtocol = securityConfig.getSecurityProtocol(); } @Override @@ -46,6 +53,7 @@ public Properties configure(ConnectorSession session) properties.setProperty(BOOTSTRAP_SERVERS_CONFIG, nodes.stream() .map(HostAddress::toString) .collect(joining(","))); + properties.setProperty(SECURITY_PROTOCOL_CONFIG, securityProtocol.name()); return properties; } } diff --git a/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/PlainTextKafkaConsumerFactory.java b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/PlainTextKafkaConsumerFactory.java index b4b0e1786593..10580d4919d5 100644 --- a/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/PlainTextKafkaConsumerFactory.java +++ b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/PlainTextKafkaConsumerFactory.java @@ -16,6 +16,7 @@ import io.airlift.units.DataSize; import io.trino.spi.HostAddress; import io.trino.spi.connector.ConnectorSession; +import org.apache.kafka.common.security.auth.SecurityProtocol; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import javax.inject.Inject; @@ -25,6 +26,7 @@ import static java.util.Objects.requireNonNull; import static java.util.stream.Collectors.joining; +import static org.apache.kafka.clients.CommonClientConfigs.SECURITY_PROTOCOL_CONFIG; import static org.apache.kafka.clients.consumer.ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG; import static org.apache.kafka.clients.consumer.ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG; import static org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG; @@ -36,14 +38,19 @@ public class PlainTextKafkaConsumerFactory { private final Set nodes; private final DataSize kafkaBufferSize; + private final SecurityProtocol securityProtocol; @Inject - public PlainTextKafkaConsumerFactory(KafkaConfig kafkaConfig) + public PlainTextKafkaConsumerFactory( + KafkaConfig kafkaConfig, + KafkaSecurityConfig securityConfig) { requireNonNull(kafkaConfig, "kafkaConfig is null"); + requireNonNull(securityConfig, "securityConfig is null"); nodes = kafkaConfig.getNodes(); kafkaBufferSize = kafkaConfig.getKafkaBufferSize(); + securityProtocol = securityConfig.getSecurityProtocol(); } @Override @@ -57,6 +64,7 @@ public Properties configure(ConnectorSession session) properties.setProperty(VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); properties.setProperty(RECEIVE_BUFFER_CONFIG, Long.toString(kafkaBufferSize.toBytes())); properties.setProperty(ENABLE_AUTO_COMMIT_CONFIG, Boolean.toString(false)); + properties.setProperty(SECURITY_PROTOCOL_CONFIG, securityProtocol.name()); return properties; } } diff --git a/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/PlainTextKafkaProducerFactory.java b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/PlainTextKafkaProducerFactory.java index cd45bdbdd69c..5fb358bbc5f0 100644 --- a/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/PlainTextKafkaProducerFactory.java +++ b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/PlainTextKafkaProducerFactory.java @@ -15,6 +15,7 @@ import io.trino.spi.HostAddress; import io.trino.spi.connector.ConnectorSession; +import org.apache.kafka.common.security.auth.SecurityProtocol; import org.apache.kafka.common.serialization.ByteArraySerializer; import javax.inject.Inject; @@ -24,6 +25,7 @@ import static java.util.Objects.requireNonNull; import static java.util.stream.Collectors.joining; +import static org.apache.kafka.clients.CommonClientConfigs.SECURITY_PROTOCOL_CONFIG; import static org.apache.kafka.clients.consumer.ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG; import static org.apache.kafka.clients.producer.ProducerConfig.ACKS_CONFIG; import static org.apache.kafka.clients.producer.ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG; @@ -34,13 +36,16 @@ public class PlainTextKafkaProducerFactory implements KafkaProducerFactory { private final Set nodes; + private final SecurityProtocol securityProtocol; @Inject - public PlainTextKafkaProducerFactory(KafkaConfig kafkaConfig) + public PlainTextKafkaProducerFactory(KafkaConfig kafkaConfig, KafkaSecurityConfig securityConfig) { requireNonNull(kafkaConfig, "kafkaConfig is null"); + requireNonNull(securityConfig, "securityConfig is null"); nodes = kafkaConfig.getNodes(); + securityProtocol = securityConfig.getSecurityProtocol(); } @Override @@ -54,6 +59,7 @@ public Properties configure(ConnectorSession session) properties.setProperty(VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); properties.setProperty(ACKS_CONFIG, "all"); properties.setProperty(LINGER_MS_CONFIG, Long.toString(5)); + properties.setProperty(SECURITY_PROTOCOL_CONFIG, securityProtocol.name()); return properties; } } diff --git a/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/SslKafkaAdminFactory.java b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/SslKafkaAdminFactory.java new file mode 100644 index 000000000000..a5e005d21d5f --- /dev/null +++ b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/SslKafkaAdminFactory.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 io.trino.plugin.kafka; + +import com.google.common.collect.ImmutableMap; +import io.trino.plugin.kafka.security.ForKafkaSsl; +import io.trino.plugin.kafka.security.KafkaSslConfig; +import io.trino.spi.connector.ConnectorSession; + +import javax.inject.Inject; + +import java.util.Properties; + +import static java.util.Objects.requireNonNull; + +public class SslKafkaAdminFactory + implements KafkaAdminFactory +{ + private final ImmutableMap map; + private final KafkaAdminFactory delegate; + + @Inject + public SslKafkaAdminFactory(@ForKafkaSsl KafkaAdminFactory delegate, KafkaSslConfig sslConfig) + { + this.delegate = requireNonNull(delegate, "delegate is null"); + requireNonNull(sslConfig, "sslConfig is null"); + + map = ImmutableMap.copyOf(sslConfig.getKafkaClientProperties()); + } + + @Override + public Properties configure(ConnectorSession session) + { + Properties properties = new Properties(); + properties.putAll(delegate.configure(session)); + properties.putAll(map); + return properties; + } +} diff --git a/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/SslKafkaConsumerFactory.java b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/SslKafkaConsumerFactory.java new file mode 100644 index 000000000000..a18058f6b61f --- /dev/null +++ b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/SslKafkaConsumerFactory.java @@ -0,0 +1,50 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.trino.plugin.kafka; + +import com.google.common.collect.ImmutableMap; +import io.trino.plugin.kafka.security.ForKafkaSsl; +import io.trino.plugin.kafka.security.KafkaSslConfig; +import io.trino.spi.connector.ConnectorSession; + +import javax.inject.Inject; + +import java.util.Properties; + +import static java.util.Objects.requireNonNull; + +public class SslKafkaConsumerFactory + implements KafkaConsumerFactory +{ + private final ImmutableMap map; + private final KafkaConsumerFactory delegate; + + @Inject + public SslKafkaConsumerFactory(@ForKafkaSsl KafkaConsumerFactory delegate, KafkaSslConfig sslConfig) + { + this.delegate = requireNonNull(delegate, "delegate is null"); + requireNonNull(sslConfig, "sslConfig is null"); + + map = ImmutableMap.copyOf(sslConfig.getKafkaClientProperties()); + } + + @Override + public Properties configure(ConnectorSession session) + { + Properties properties = new Properties(); + properties.putAll(delegate.configure(session)); + properties.putAll(map); + return properties; + } +} diff --git a/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/SslKafkaProducerFactory.java b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/SslKafkaProducerFactory.java new file mode 100644 index 000000000000..0495e2f096f0 --- /dev/null +++ b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/SslKafkaProducerFactory.java @@ -0,0 +1,50 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.trino.plugin.kafka; + +import com.google.common.collect.ImmutableMap; +import io.trino.plugin.kafka.security.ForKafkaSsl; +import io.trino.plugin.kafka.security.KafkaSslConfig; +import io.trino.spi.connector.ConnectorSession; + +import javax.inject.Inject; + +import java.util.Properties; + +import static java.util.Objects.requireNonNull; + +public class SslKafkaProducerFactory + implements KafkaProducerFactory +{ + private final ImmutableMap map; + private final KafkaProducerFactory delegate; + + @Inject + public SslKafkaProducerFactory(@ForKafkaSsl KafkaProducerFactory delegate, KafkaSslConfig sslConfig) + { + this.delegate = requireNonNull(delegate, "delegate is null"); + requireNonNull(sslConfig, "sslConfig is null"); + + map = ImmutableMap.copyOf(sslConfig.getKafkaClientProperties()); + } + + @Override + public Properties configure(ConnectorSession session) + { + Properties properties = new Properties(); + properties.putAll(delegate.configure(session)); + properties.putAll(map); + return properties; + } +} diff --git a/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/security/ForKafkaSsl.java b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/security/ForKafkaSsl.java new file mode 100644 index 000000000000..4f4b95414e8b --- /dev/null +++ b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/security/ForKafkaSsl.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 io.trino.plugin.kafka.security; + +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 ForKafkaSsl +{ +} diff --git a/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/security/KafkaEndpointIdentificationAlgorithm.java b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/security/KafkaEndpointIdentificationAlgorithm.java new file mode 100644 index 000000000000..74b2aab83a84 --- /dev/null +++ b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/security/KafkaEndpointIdentificationAlgorithm.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 io.trino.plugin.kafka.security; + +import java.util.Optional; + +import static java.util.Locale.ENGLISH; + +public enum KafkaEndpointIdentificationAlgorithm +{ + HTTPS("https"), + DISABLED(""); + + private final String value; + + KafkaEndpointIdentificationAlgorithm(String value) + { + this.value = value; + } + + public static Optional fromString(String value) + { + return Optional.of(KafkaEndpointIdentificationAlgorithm.valueOf(value.toUpperCase(ENGLISH))); + } + + public String getValue() + { + return value; + } +} diff --git a/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/KafkaProducerModule.java b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/security/KafkaKeystoreTruststoreType.java similarity index 62% rename from plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/KafkaProducerModule.java rename to plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/security/KafkaKeystoreTruststoreType.java index af1954cc3dfd..eecaea983357 100644 --- a/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/KafkaProducerModule.java +++ b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/security/KafkaKeystoreTruststoreType.java @@ -11,18 +11,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.kafka; +package io.trino.plugin.kafka.security; -import com.google.inject.Binder; -import com.google.inject.Module; -import com.google.inject.Scopes; +import java.util.Optional; -public class KafkaProducerModule - implements Module +import static java.util.Locale.ENGLISH; + +public enum KafkaKeystoreTruststoreType { - @Override - public void configure(Binder binder) + JKS, + PKCS12; + + public static Optional fromString(String value) { - binder.bind(KafkaProducerFactory.class).to(PlainTextKafkaProducerFactory.class).in(Scopes.SINGLETON); + return Optional.of(KafkaKeystoreTruststoreType.valueOf(value.toUpperCase(ENGLISH))); } } diff --git a/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/security/KafkaSecurityModule.java b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/security/KafkaSecurityModule.java new file mode 100644 index 000000000000..d1cdd46c4982 --- /dev/null +++ b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/security/KafkaSecurityModule.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 io.trino.plugin.kafka.security; + +import com.google.inject.Binder; +import com.google.inject.Module; +import io.airlift.configuration.AbstractConfigurationAwareModule; +import io.trino.plugin.kafka.KafkaSecurityConfig; +import org.apache.kafka.common.security.auth.SecurityProtocol; + +import static io.airlift.configuration.ConditionalModule.installModuleIf; +import static io.airlift.configuration.ConfigurationModule.installModules; + +public class KafkaSecurityModule + extends AbstractConfigurationAwareModule +{ + @Override + protected void setup(Binder binder) + { + bindSecurityModule( + SecurityProtocol.SSL, + installModules(new SslSecurityModule())); + } + + private void bindSecurityModule(SecurityProtocol securityProtocol, Module module) + { + install(installModuleIf( + KafkaSecurityConfig.class, + config -> config.getSecurityProtocol().equals(securityProtocol), + module)); + } +} diff --git a/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/security/KafkaSslConfig.java b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/security/KafkaSslConfig.java new file mode 100644 index 000000000000..af379ea624ce --- /dev/null +++ b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/security/KafkaSslConfig.java @@ -0,0 +1,187 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.trino.plugin.kafka.security; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.inject.ConfigurationException; +import com.google.inject.spi.Message; +import io.airlift.configuration.Config; +import io.airlift.configuration.ConfigDescription; +import io.airlift.configuration.ConfigSecuritySensitive; +import io.airlift.configuration.validation.FileExists; + +import javax.annotation.PostConstruct; + +import java.util.Map; +import java.util.Optional; + +import static io.trino.plugin.kafka.security.KafkaEndpointIdentificationAlgorithm.HTTPS; +import static io.trino.plugin.kafka.security.KafkaKeystoreTruststoreType.JKS; +import static org.apache.kafka.common.config.SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG; +import static org.apache.kafka.common.config.SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG; +import static org.apache.kafka.common.config.SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG; +import static org.apache.kafka.common.config.SslConfigs.SSL_KEYSTORE_TYPE_CONFIG; +import static org.apache.kafka.common.config.SslConfigs.SSL_KEY_PASSWORD_CONFIG; +import static org.apache.kafka.common.config.SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG; +import static org.apache.kafka.common.config.SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG; +import static org.apache.kafka.common.config.SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG; + +/** + * {@KafkaSslConfig} manages Kafka SSL authentication and encryption between clients and brokers. + */ +public class KafkaSslConfig +{ + private String keystoreLocation; + private String keystorePassword; + private KafkaKeystoreTruststoreType keystoreType = JKS; + private String truststoreLocation; + private String truststorePassword; + private KafkaKeystoreTruststoreType truststoreType = JKS; + private String keyPassword; + private KafkaEndpointIdentificationAlgorithm endpointIdentificationAlgorithm = HTTPS; + + public Optional<@FileExists String> getKeystoreLocation() + { + return Optional.ofNullable(keystoreLocation); + } + + @Config("kafka.ssl.keystore.location") + @ConfigDescription("The location of the key store file. This can be used for two-way authentication for client") + public KafkaSslConfig setKeystoreLocation(String keystoreLocation) + { + this.keystoreLocation = keystoreLocation; + return this; + } + + public Optional getKeystorePassword() + { + return Optional.ofNullable(keystorePassword); + } + + @Config("kafka.ssl.keystore.password") + @ConfigDescription("The store password for the key store file") + @ConfigSecuritySensitive + public KafkaSslConfig setKeystorePassword(String keystorePassword) + { + this.keystorePassword = keystorePassword; + return this; + } + + public Optional getKeystoreType() + { + return Optional.ofNullable(keystoreType); + } + + @Config("kafka.ssl.keystore.type") + @ConfigDescription("The file format of the key store file") + public KafkaSslConfig setKeystoreType(KafkaKeystoreTruststoreType keystoreType) + { + this.keystoreType = keystoreType; + return this; + } + + public Optional<@FileExists String> getTruststoreLocation() + { + return Optional.ofNullable(truststoreLocation); + } + + @Config("kafka.ssl.truststore.location") + @ConfigDescription("The location of the trust store file") + public KafkaSslConfig setTruststoreLocation(String truststoreLocation) + { + this.truststoreLocation = truststoreLocation; + return this; + } + + public Optional getTruststorePassword() + { + return Optional.ofNullable(truststorePassword); + } + + @Config("kafka.ssl.truststore.password") + @ConfigDescription("The password for the trust store file") + @ConfigSecuritySensitive + public KafkaSslConfig setTruststorePassword(String truststorePassword) + { + this.truststorePassword = truststorePassword; + return this; + } + + public Optional getTruststoreType() + { + return Optional.ofNullable(truststoreType); + } + + @Config("kafka.ssl.truststore.type") + @ConfigDescription("The file format of the trust store file") + public KafkaSslConfig setTruststoreType(KafkaKeystoreTruststoreType truststoreType) + { + this.truststoreType = truststoreType; + return this; + } + + public Optional getKeyPassword() + { + return Optional.ofNullable(keyPassword); + } + + @Config("kafka.ssl.key.password") + @ConfigDescription("The password of the private key in the key store file") + @ConfigSecuritySensitive + public KafkaSslConfig setKeyPassword(String keyPassword) + { + this.keyPassword = keyPassword; + return this; + } + + public Optional getEndpointIdentificationAlgorithm() + { + return Optional.ofNullable(endpointIdentificationAlgorithm); + } + + @Config("kafka.ssl.endpoint-identification-algorithm") + @ConfigDescription("The endpoint identification algorithm to validate server hostname using server certificate") + public KafkaSslConfig setEndpointIdentificationAlgorithm(KafkaEndpointIdentificationAlgorithm endpointIdentificationAlgorithm) + { + this.endpointIdentificationAlgorithm = endpointIdentificationAlgorithm; + return this; + } + + public Map getKafkaClientProperties() + { + ImmutableMap.Builder properties = ImmutableMap.builder(); + getKeystoreLocation().ifPresent(v -> properties.put(SSL_KEYSTORE_LOCATION_CONFIG, v)); + getKeystorePassword().ifPresent(v -> properties.put(SSL_KEYSTORE_PASSWORD_CONFIG, v)); + getKeystoreType().ifPresent(v -> properties.put(SSL_KEYSTORE_TYPE_CONFIG, v.name())); + getTruststoreLocation().ifPresent(v -> properties.put(SSL_TRUSTSTORE_LOCATION_CONFIG, v)); + getTruststorePassword().ifPresent(v -> properties.put(SSL_TRUSTSTORE_PASSWORD_CONFIG, v)); + getTruststoreType().ifPresent(v -> properties.put(SSL_TRUSTSTORE_TYPE_CONFIG, v.name())); + getKeyPassword().ifPresent(v -> properties.put(SSL_KEY_PASSWORD_CONFIG, v)); + getEndpointIdentificationAlgorithm().ifPresent(v -> properties.put(SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, v.getValue())); + + return properties.build(); + } + + @PostConstruct + public void validate() + { + if (getKeystoreLocation().isPresent() && getKeystorePassword().isEmpty()) { + throw new ConfigurationException(ImmutableList.of(new Message("kafka.ssl.keystore.password must set when kafka.ssl.keystore.location is given"))); + } + if (getTruststoreLocation().isPresent() && getTruststorePassword().isEmpty()) { + throw new ConfigurationException(ImmutableList.of(new Message("kafka.ssl.truststore.password must set when kafka.ssl.truststore.location is given"))); + } + } +} diff --git a/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/KafkaAdminModule.java b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/security/SslSecurityModule.java similarity index 78% rename from plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/KafkaAdminModule.java rename to plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/security/SslSecurityModule.java index b4bc52bb21e5..3d070c132d8d 100644 --- a/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/KafkaAdminModule.java +++ b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/security/SslSecurityModule.java @@ -11,18 +11,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.kafka; +package io.trino.plugin.kafka.security; import com.google.inject.Binder; import com.google.inject.Module; -import com.google.inject.Scopes; -public class KafkaAdminModule +import static io.airlift.configuration.ConfigBinder.configBinder; + +public class SslSecurityModule implements Module { @Override public void configure(Binder binder) { - binder.bind(KafkaAdminFactory.class).to(PlainTextKafkaAdminFactory.class).in(Scopes.SINGLETON); + configBinder(binder).bindConfig(KafkaSslConfig.class); } } diff --git a/plugin/trino-kafka/src/test/java/io/trino/plugin/kafka/TestKafkaPlugin.java b/plugin/trino-kafka/src/test/java/io/trino/plugin/kafka/TestKafkaPlugin.java index bf453b946944..18dc2b8b24ec 100644 --- a/plugin/trino-kafka/src/test/java/io/trino/plugin/kafka/TestKafkaPlugin.java +++ b/plugin/trino-kafka/src/test/java/io/trino/plugin/kafka/TestKafkaPlugin.java @@ -19,8 +19,15 @@ import io.trino.testing.TestingConnectorContext; import org.testng.annotations.Test; +import java.io.FileWriter; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; + import static com.google.common.collect.Iterables.getOnlyElement; import static io.airlift.testing.Assertions.assertInstanceOf; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.testng.Assert.assertNotNull; public class TestKafkaPlugin @@ -43,4 +50,108 @@ public void testSpinup() assertNotNull(connector); connector.shutdown(); } + + @Test + public void testSslSpinup() + throws IOException + { + KafkaPlugin plugin = new KafkaPlugin(); + + ConnectorFactory factory = getOnlyElement(plugin.getConnectorFactories()); + assertInstanceOf(factory, KafkaConnectorFactory.class); + + String secret = "confluent"; + Path keystorePath = Files.createTempFile("keystore", ".jks"); + Path truststorePath = Files.createTempFile("truststore", ".jks"); + + writeToFile(keystorePath, secret); + writeToFile(truststorePath, secret); + + Connector connector = factory.create( + "test-connector", + ImmutableMap.builder() + .put("kafka.table-names", "test") + .put("kafka.nodes", "localhost:9092") + .put("kafka.security-protocol", "SSL") + .put("kafka.ssl.keystore.type", "JKS") + .put("kafka.ssl.keystore.location", keystorePath.toString()) + .put("kafka.ssl.keystore.password", "keystore-password") + .put("kafka.ssl.key.password", "key-password") + .put("kafka.ssl.truststore.type", "JKS") + .put("kafka.ssl.truststore.location", truststorePath.toString()) + .put("kafka.ssl.truststore.password", "truststore-password") + .put("kafka.ssl.endpoint-identification-algorithm", "https") + .build(), + new TestingConnectorContext()); + assertNotNull(connector); + connector.shutdown(); + } + + @Test + public void testSslKeystoreMissingFileSpindown() + throws IOException + { + KafkaPlugin plugin = new KafkaPlugin(); + + ConnectorFactory factory = getOnlyElement(plugin.getConnectorFactories()); + assertInstanceOf(factory, KafkaConnectorFactory.class); + + Path truststorePath = Files.createTempFile("test", ".jks"); + + assertThatThrownBy(() -> factory.create( + "test-connector", + ImmutableMap.builder() + .put("kafka.table-names", "test") + .put("kafka.nodes", "localhost:9092") + .put("kafka.security-protocol", "SSL") + .put("kafka.ssl.keystore.type", "JKS") + .put("kafka.ssl.keystore.location", "/not/a/real/path") + .put("kafka.ssl.keystore.password", "keystore-password") + .put("kafka.ssl.key.password", "key-password") + .put("kafka.ssl.truststore.type", "JKS") + .put("kafka.ssl.truststore.location", truststorePath.toString()) + .put("kafka.ssl.truststore.password", "truststore-password") + .put("kafka.ssl.endpoint-identification-algorithm", "https") + .build(), + new TestingConnectorContext())) + .hasMessageContaining("Error: Invalid configuration property kafka.ssl.keystore.location: file does not exist: /not/a/real/path"); + } + + @Test + public void testSslTruststoreMissingFileSpindown() + throws IOException + { + KafkaPlugin plugin = new KafkaPlugin(); + + ConnectorFactory factory = getOnlyElement(plugin.getConnectorFactories()); + assertInstanceOf(factory, KafkaConnectorFactory.class); + + Path keystorePath = Files.createTempFile("test", ".jks"); + + assertThatThrownBy(() -> factory.create( + "test-connector", + ImmutableMap.builder() + .put("kafka.table-names", "test") + .put("kafka.nodes", "localhost:9092") + .put("kafka.security-protocol", "SSL") + .put("kafka.ssl.keystore.type", "JKS") + .put("kafka.ssl.keystore.location", keystorePath.toString()) + .put("kafka.ssl.keystore.password", "keystore-password") + .put("kafka.ssl.key.password", "key-password") + .put("kafka.ssl.truststore.type", "JKS") + .put("kafka.ssl.truststore.location", "/not/a/real/path") + .put("kafka.ssl.truststore.password", "truststore-password") + .put("kafka.ssl.endpoint-identification-algorithm", "https") + .build(), + new TestingConnectorContext())) + .hasMessageContaining("Error: Invalid configuration property kafka.ssl.truststore.location: file does not exist: /not/a/real/path"); + } + + private void writeToFile(Path filepath, String content) + throws IOException + { + try (FileWriter writer = new FileWriter(filepath.toFile(), StandardCharsets.UTF_8)) { + writer.write(content); + } + } } diff --git a/plugin/trino-kafka/src/test/java/io/trino/plugin/kafka/TestKafkaSecurityConfig.java b/plugin/trino-kafka/src/test/java/io/trino/plugin/kafka/TestKafkaSecurityConfig.java new file mode 100644 index 000000000000..c4ba1284b19d --- /dev/null +++ b/plugin/trino-kafka/src/test/java/io/trino/plugin/kafka/TestKafkaSecurityConfig.java @@ -0,0 +1,83 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.kafka; + +import com.google.common.collect.ImmutableMap; +import org.apache.kafka.common.security.auth.SecurityProtocol; +import org.testng.annotations.DataProvider; +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; +import static org.apache.kafka.common.security.auth.SecurityProtocol.PLAINTEXT; +import static org.apache.kafka.common.security.auth.SecurityProtocol.SASL_PLAINTEXT; +import static org.apache.kafka.common.security.auth.SecurityProtocol.SASL_SSL; +import static org.apache.kafka.common.security.auth.SecurityProtocol.SSL; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +public class TestKafkaSecurityConfig +{ + @Test + public void testDefaults() + { + assertRecordedDefaults(recordDefaults(KafkaSecurityConfig.class) + .setSecurityProtocol(PLAINTEXT)); + } + + @Test + public void testExplicitPropertyMappings() + { + Map properties = new ImmutableMap.Builder() + .put("kafka.security-protocol", "SSL") + .build(); + + KafkaSecurityConfig expected = new KafkaSecurityConfig() + .setSecurityProtocol(SSL); + + assertFullMapping(properties, expected); + } + + @Test(dataProvider = "validSecurityProtocols") + public void testValidSecurityProtocols(SecurityProtocol securityProtocol) + { + new KafkaSecurityConfig() + .setSecurityProtocol(securityProtocol) + .validate(); + } + + @DataProvider(name = "validSecurityProtocols") + public Object[][] validSecurityProtocols() + { + return new Object[][] {{PLAINTEXT}, {SSL}}; + } + + @Test(dataProvider = "invalidSecurityProtocols") + public void testInvalidSecurityProtocol(SecurityProtocol securityProtocol) + { + assertThatThrownBy(() -> new KafkaSecurityConfig() + .setSecurityProtocol(securityProtocol) + .validate()) + .isInstanceOf(IllegalStateException.class) + .hasMessage("Only PLAINTEXT and SSL security protocols are supported"); + } + + @DataProvider(name = "invalidSecurityProtocols") + public Object[][] invalidSecurityProtocols() + { + return new Object[][] {{SASL_PLAINTEXT}, {SASL_SSL}}; + } +} diff --git a/plugin/trino-kafka/src/test/java/io/trino/plugin/kafka/TestKafkaSslConfig.java b/plugin/trino-kafka/src/test/java/io/trino/plugin/kafka/TestKafkaSslConfig.java new file mode 100644 index 000000000000..d8cf8e912dda --- /dev/null +++ b/plugin/trino-kafka/src/test/java/io/trino/plugin/kafka/TestKafkaSslConfig.java @@ -0,0 +1,176 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.trino.plugin.kafka; + +import com.google.common.collect.ImmutableMap; +import com.google.inject.ConfigurationException; +import io.trino.plugin.kafka.security.KafkaEndpointIdentificationAlgorithm; +import io.trino.plugin.kafka.security.KafkaSslConfig; +import org.testng.annotations.Test; + +import java.io.FileWriter; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +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; +import static io.trino.plugin.kafka.security.KafkaEndpointIdentificationAlgorithm.DISABLED; +import static io.trino.plugin.kafka.security.KafkaEndpointIdentificationAlgorithm.HTTPS; +import static io.trino.plugin.kafka.security.KafkaKeystoreTruststoreType.JKS; +import static io.trino.plugin.kafka.security.KafkaKeystoreTruststoreType.PKCS12; +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.apache.kafka.common.config.SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG; +import static org.apache.kafka.common.config.SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG; +import static org.apache.kafka.common.config.SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG; +import static org.apache.kafka.common.config.SslConfigs.SSL_KEYSTORE_TYPE_CONFIG; +import static org.apache.kafka.common.config.SslConfigs.SSL_KEY_PASSWORD_CONFIG; +import static org.apache.kafka.common.config.SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG; +import static org.apache.kafka.common.config.SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG; +import static org.apache.kafka.common.config.SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +public class TestKafkaSslConfig +{ + @Test + public void testDefaults() + { + assertRecordedDefaults(recordDefaults(KafkaSslConfig.class) + .setKeystoreLocation(null) + .setKeystorePassword(null) + .setKeystoreType(JKS) + .setTruststoreLocation(null) + .setTruststorePassword(null) + .setTruststoreType(JKS) + .setKeyPassword(null) + .setEndpointIdentificationAlgorithm(HTTPS)); + } + + @Test + public void testExplicitPropertyMappings() + throws IOException + { + String secret = "confluent"; + Path keystorePath = Files.createTempFile("keystore", ".p12"); + Path truststorePath = Files.createTempFile("truststore", ".p12"); + + writeToFile(keystorePath, secret); + writeToFile(truststorePath, secret); + + Map properties = new ImmutableMap.Builder() + .put("kafka.ssl.keystore.location", keystorePath.toString()) + .put("kafka.ssl.keystore.password", "keystore-password") + .put("kafka.ssl.keystore.type", "PKCS12") + .put("kafka.ssl.truststore.location", truststorePath.toString()) + .put("kafka.ssl.truststore.password", "truststore-password") + .put("kafka.ssl.truststore.type", "PKCS12") + .put("kafka.ssl.key.password", "key-password") + .put("kafka.ssl.endpoint-identification-algorithm", "disabled") + .build(); + KafkaSslConfig expected = new KafkaSslConfig() + .setKeystoreLocation(keystorePath.toString()) + .setKeystorePassword("keystore-password") + .setKeystoreType(PKCS12) + .setTruststoreLocation(truststorePath.toString()) + .setTruststorePassword("truststore-password") + .setTruststoreType(PKCS12) + .setKeyPassword("key-password") + .setEndpointIdentificationAlgorithm(DISABLED); + + assertFullMapping(properties, expected); + } + + @Test + public void testAllConfigPropertiesAreContained() + { + KafkaSslConfig config = new KafkaSslConfig() + .setKeystoreLocation("/some/path/to/keystore") + .setKeystorePassword("superSavePasswordForKeystore") + .setKeystoreType(JKS) + .setTruststoreLocation("/some/path/to/truststore") + .setTruststorePassword("superSavePasswordForTruststore") + .setTruststoreType(JKS) + .setKeyPassword("aSslKeyPassword") + .setEndpointIdentificationAlgorithm(HTTPS); + + Map securityProperties = config.getKafkaClientProperties(); + // Since security related properties are all passed to the underlying kafka-clients library, + // the property names must match those expected by kafka-clients + assertThat(securityProperties) + .containsExactlyInAnyOrderEntriesOf(ImmutableMap.copyOf(Map.of( + SSL_KEYSTORE_LOCATION_CONFIG, "/some/path/to/keystore", + SSL_KEYSTORE_PASSWORD_CONFIG, "superSavePasswordForKeystore", + SSL_KEYSTORE_TYPE_CONFIG, JKS.name(), + SSL_TRUSTSTORE_LOCATION_CONFIG, "/some/path/to/truststore", + SSL_TRUSTSTORE_PASSWORD_CONFIG, "superSavePasswordForTruststore", + SSL_TRUSTSTORE_TYPE_CONFIG, JKS.name(), + SSL_KEY_PASSWORD_CONFIG, "aSslKeyPassword", + SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, HTTPS.getValue()))); + } + + @Test + public void testDisabledEndpointIdentificationAlgorithm() + { + KafkaSslConfig config = new KafkaSslConfig(); + if (KafkaEndpointIdentificationAlgorithm.fromString("disabled").isPresent()) { + config.setEndpointIdentificationAlgorithm(KafkaEndpointIdentificationAlgorithm.fromString("disabled").get()); + } + Map securityProperties = config.getKafkaClientProperties(); + assertThat(securityProperties).containsKey(SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG); + assertThat(securityProperties.get(SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG)).isEqualTo(""); + } + + @Test + public void testFailOnMissingKeystorePasswordWithKeystoreLocationSet() + throws Exception + { + String secret = "confluent"; + Path keystorePath = Files.createTempFile("keystore", ".p12"); + + writeToFile(keystorePath, secret); + + KafkaSslConfig config = new KafkaSslConfig(); + config.setKeystoreLocation(keystorePath.toString()); + assertThatThrownBy(config::validate) + .isInstanceOf(ConfigurationException.class) + .hasMessageContaining("kafka.ssl.keystore.password must set when kafka.ssl.keystore.location is given"); + } + + @Test + public void testFailOnMissingTruststorePasswordWithTruststoreLocationSet() + throws Exception + { + String secret = "confluent"; + Path truststorePath = Files.createTempFile("truststore", ".p12"); + + writeToFile(truststorePath, secret); + + KafkaSslConfig config = new KafkaSslConfig(); + config.setTruststoreLocation(truststorePath.toString()); + assertThatThrownBy(config::validate) + .isInstanceOf(ConfigurationException.class) + .hasMessageContaining("kafka.ssl.truststore.password must set when kafka.ssl.truststore.location is given"); + } + + private void writeToFile(Path filepath, String content) + throws IOException + { + try (FileWriter writer = new FileWriter(filepath.toFile(), UTF_8)) { + writer.write(content); + } + } +} diff --git a/plugin/trino-kinesis/pom.xml b/plugin/trino-kinesis/pom.xml index 6a64cd7edf18..c3400ef5cab1 100644 --- a/plugin/trino-kinesis/pom.xml +++ b/plugin/trino-kinesis/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-kudu/pom.xml b/plugin/trino-kudu/pom.xml index 25f8d217e85a..e7ac7c3d3197 100644 --- a/plugin/trino-kudu/pom.xml +++ b/plugin/trino-kudu/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-local-file/pom.xml b/plugin/trino-local-file/pom.xml index ca1c1a7eb5b4..97e75a7cddb3 100644 --- a/plugin/trino-local-file/pom.xml +++ b/plugin/trino-local-file/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-memory/pom.xml b/plugin/trino-memory/pom.xml index b7b04c731f64..3750295a7c00 100644 --- a/plugin/trino-memory/pom.xml +++ b/plugin/trino-memory/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-memsql/pom.xml b/plugin/trino-memsql/pom.xml index 7643c4ecc227..e75be50df81e 100644 --- a/plugin/trino-memsql/pom.xml +++ b/plugin/trino-memsql/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-memsql/src/test/java/io/trino/plugin/memsql/TestMemSqlTypeMapping.java b/plugin/trino-memsql/src/test/java/io/trino/plugin/memsql/TestMemSqlTypeMapping.java index 91c14d47292a..1ee621efe29d 100644 --- a/plugin/trino-memsql/src/test/java/io/trino/plugin/memsql/TestMemSqlTypeMapping.java +++ b/plugin/trino-memsql/src/test/java/io/trino/plugin/memsql/TestMemSqlTypeMapping.java @@ -147,6 +147,7 @@ private static DataTypeTest doublePrecisionFloatingPointTests(DataType d // we are not testing Nan/-Infinity/+Infinity as those are not supported by MemSQL return DataTypeTest.create() .addRoundTrip(doubleType, 1.0e100d) + .addRoundTrip(doubleType, 123.456E10) .addRoundTrip(doubleType, null); } diff --git a/plugin/trino-ml/pom.xml b/plugin/trino-ml/pom.xml index ed8527d23142..f412149570e8 100644 --- a/plugin/trino-ml/pom.xml +++ b/plugin/trino-ml/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-mongodb/pom.xml b/plugin/trino-mongodb/pom.xml index daac59775d6c..f7b6466514f1 100644 --- a/plugin/trino-mongodb/pom.xml +++ b/plugin/trino-mongodb/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-mysql/pom.xml b/plugin/trino-mysql/pom.xml index 463e7ef71508..af55465b9d74 100644 --- a/plugin/trino-mysql/pom.xml +++ b/plugin/trino-mysql/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-mysql/src/test/java/io/trino/plugin/mysql/TestMySqlTypeMapping.java b/plugin/trino-mysql/src/test/java/io/trino/plugin/mysql/TestMySqlTypeMapping.java index 0331903f5148..15d6a80219de 100644 --- a/plugin/trino-mysql/src/test/java/io/trino/plugin/mysql/TestMySqlTypeMapping.java +++ b/plugin/trino-mysql/src/test/java/io/trino/plugin/mysql/TestMySqlTypeMapping.java @@ -819,6 +819,7 @@ private static DataTypeTest doublePrecisionFloatingPointTests(DataType d // we are not testing Nan/-Infinity/+Infinity as those are not supported by MySQL return DataTypeTest.create() .addRoundTrip(doubleType, 1.0e100d) + .addRoundTrip(doubleType, 123.456E10) .addRoundTrip(doubleType, null); } diff --git a/plugin/trino-oracle/pom.xml b/plugin/trino-oracle/pom.xml index 8cb0718b7287..f0d62ae0c4f9 100644 --- a/plugin/trino-oracle/pom.xml +++ b/plugin/trino-oracle/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/AbstractTestOracleTypeMapping.java b/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/AbstractTestOracleTypeMapping.java index f0d01842ea27..d6fb13b61a27 100644 --- a/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/AbstractTestOracleTypeMapping.java +++ b/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/AbstractTestOracleTypeMapping.java @@ -147,6 +147,7 @@ public void testFloatingPointMappings() .addRoundTrip("real", "-infinity()", REAL, "CAST(-infinity() AS real)") .addRoundTrip("real", "NULL", REAL, "CAST(NULL AS real)") .addRoundTrip("double", "1.0E100", DOUBLE, "double '1.0E100'") + .addRoundTrip("double", "123.456E10", DOUBLE, "123.456E10") .addRoundTrip("double", "nan()", DOUBLE, "CAST(nan() AS double)") .addRoundTrip("double", "+infinity()", DOUBLE, "CAST(+infinity() AS double)") .addRoundTrip("double", "-infinity()", DOUBLE, "CAST(-infinity() AS double)") diff --git a/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/TestOracleCaseInsensitiveMapping.java b/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/TestOracleCaseInsensitiveMapping.java index 726ee85d7eec..09e7cc62528f 100644 --- a/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/TestOracleCaseInsensitiveMapping.java +++ b/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/TestOracleCaseInsensitiveMapping.java @@ -126,7 +126,7 @@ public void testSchemaNameClash() try (AutoCloseable ignore1 = withSchema(schemaName); AutoCloseable ignore2 = withSchema(otherSchemaName); AutoCloseable ignore3 = withTable(schemaName + ".some_table_name", "(c varchar(5))")) { - assertThat(computeActual("SHOW SCHEMAS").getOnlyColumn().filter("casesensitivename"::equals)).hasSize(1); // TODO change io.prestosql.plugin.jdbc.JdbcClient.getSchemaNames to return a List + assertThat(computeActual("SHOW SCHEMAS").getOnlyColumn().filter("casesensitivename"::equals)).hasSize(1); // TODO change io.trino.plugin.jdbc.JdbcClient.getSchemaNames to return a List assertQueryFails("SHOW TABLES FROM casesensitivename", "Failed to find remote schema name:.*Multiple entries with same key.*"); assertQueryFails("SELECT * FROM casesensitivename.some_table_name", "Failed to find remote schema name:.*Multiple entries with same key.*"); } diff --git a/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/TestOracleConfig.java b/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/TestOracleConfig.java index 015c27c43c91..ea2d60adbb0b 100644 --- a/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/TestOracleConfig.java +++ b/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/TestOracleConfig.java @@ -27,6 +27,7 @@ import static io.airlift.configuration.testing.ConfigAssertions.assertRecordedDefaults; import static io.airlift.configuration.testing.ConfigAssertions.recordDefaults; import static io.airlift.testing.ValidationAssertions.assertFailsValidation; +import static java.util.concurrent.TimeUnit.MINUTES; import static java.util.concurrent.TimeUnit.SECONDS; public class TestOracleConfig @@ -42,7 +43,7 @@ public void testDefaults() .setConnectionPoolEnabled(true) .setConnectionPoolMinSize(1) .setConnectionPoolMaxSize(30) - .setInactiveConnectionTimeout(Duration.valueOf("20m"))); + .setInactiveConnectionTimeout(new Duration(20, MINUTES))); } @Test diff --git a/plugin/trino-password-authenticators/pom.xml b/plugin/trino-password-authenticators/pom.xml index 152c5cd45152..5f41100593fc 100644 --- a/plugin/trino-password-authenticators/pom.xml +++ b/plugin/trino-password-authenticators/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-phoenix/pom.xml b/plugin/trino-phoenix/pom.xml index 62a8ce620ad7..cbda167f12fb 100644 --- a/plugin/trino-phoenix/pom.xml +++ b/plugin/trino-phoenix/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml @@ -74,6 +74,11 @@ guice + + javax.annotation + javax.annotation-api + + javax.inject javax.inject diff --git a/plugin/trino-phoenix/src/main/java/io/trino/plugin/phoenix/PhoenixClient.java b/plugin/trino-phoenix/src/main/java/io/trino/plugin/phoenix/PhoenixClient.java index c8648ee054b4..0ab8755a3b2d 100644 --- a/plugin/trino-phoenix/src/main/java/io/trino/plugin/phoenix/PhoenixClient.java +++ b/plugin/trino-phoenix/src/main/java/io/trino/plugin/phoenix/PhoenixClient.java @@ -17,10 +17,12 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import io.trino.plugin.jdbc.BaseJdbcClient; +import io.trino.plugin.jdbc.BaseJdbcClient.TopNFunction; import io.trino.plugin.jdbc.ColumnMapping; import io.trino.plugin.jdbc.ConnectionFactory; import io.trino.plugin.jdbc.JdbcColumnHandle; import io.trino.plugin.jdbc.JdbcOutputTableHandle; +import io.trino.plugin.jdbc.JdbcSortItem; import io.trino.plugin.jdbc.JdbcSplit; import io.trino.plugin.jdbc.JdbcTableHandle; import io.trino.plugin.jdbc.JdbcTypeHandle; @@ -184,6 +186,7 @@ public class PhoenixClient extends BaseJdbcClient { private static final String ROWKEY = "ROWKEY"; + private static final long MAX_TOPN_LIMIT = 2000000; private final Configuration configuration; @@ -243,15 +246,12 @@ protected Collection listSchemas(Connection connection) public PreparedStatement buildSql(ConnectorSession session, Connection connection, JdbcSplit split, JdbcTableHandle table, List columnHandles) throws SQLException { - PreparedQuery preparedQuery = prepareQuery( + PreparedStatement query = prepareStatement( session, connection, table, - Optional.empty(), columnHandles, - ImmutableMap.of(), Optional.of(split)); - PreparedStatement query = new QueryBuilder(this).prepareStatement(session, connection, preparedQuery); QueryPlan queryPlan = getQueryPlan((PhoenixPreparedStatement) query); ResultSet resultSet = getResultSet(((PhoenixSplit) split).getPhoenixInputSplit(), queryPlan); return new DelegatePreparedStatement(query) @@ -264,6 +264,52 @@ public ResultSet executeQuery() }; } + public PreparedStatement prepareStatement( + ConnectorSession session, + Connection connection, + JdbcTableHandle table, + List columns, + Optional split) + throws SQLException + { + PreparedQuery preparedQuery = prepareQuery( + session, + connection, + table, + Optional.empty(), + columns, + ImmutableMap.of(), + split); + return new QueryBuilder(this).prepareStatement(session, connection, preparedQuery); + } + + @Override + public boolean supportsTopN(ConnectorSession session, JdbcTableHandle handle, List sortOrder) + { + return true; + } + + @Override + protected Optional topNFunction() + { + return Optional.of((query, sortItems, limit) -> { + // TODO: Remove when this is fixed in Phoenix. + // Phoenix severely over-estimates the memory + // required to execute a topN query. + // https://issues.apache.org/jira/browse/PHOENIX-6436 + if (limit > MAX_TOPN_LIMIT) { + return query; + } + return TopNFunction.sqlStandard(this::quoted).apply(query, sortItems, limit); + }); + } + + @Override + public boolean isTopNLimitGuaranteed(ConnectorSession session) + { + return false; + } + @Override protected Optional> limitFunction() { @@ -273,7 +319,6 @@ protected Optional> limitFunction() @Override public boolean isLimitGuaranteed(ConnectorSession session) { - // Note that limit exceeding Integer.MAX_VALUE gets completely ignored. return false; } diff --git a/plugin/trino-phoenix/src/main/java/io/trino/plugin/phoenix/PhoenixClientModule.java b/plugin/trino-phoenix/src/main/java/io/trino/plugin/phoenix/PhoenixClientModule.java index e3b41b51cf49..882864fe9867 100644 --- a/plugin/trino-phoenix/src/main/java/io/trino/plugin/phoenix/PhoenixClientModule.java +++ b/plugin/trino-phoenix/src/main/java/io/trino/plugin/phoenix/PhoenixClientModule.java @@ -27,6 +27,7 @@ import io.trino.plugin.base.util.LoggingInvocationHandler; import io.trino.plugin.jdbc.ConnectionFactory; import io.trino.plugin.jdbc.DriverConnectionFactory; +import io.trino.plugin.jdbc.ForRecordCursor; import io.trino.plugin.jdbc.ForwardingJdbcClient; import io.trino.plugin.jdbc.JdbcClient; import io.trino.plugin.jdbc.JdbcMetadataConfig; @@ -49,13 +50,17 @@ import org.apache.phoenix.jdbc.PhoenixDriver; import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver; +import javax.annotation.PreDestroy; + import java.sql.DriverManager; import java.sql.SQLException; import java.util.Map; import java.util.Properties; +import java.util.concurrent.ExecutorService; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.reflect.Reflection.newProxy; +import static com.google.common.util.concurrent.MoreExecutors.newDirectExecutorService; import static com.google.inject.multibindings.OptionalBinder.newOptionalBinder; import static io.airlift.configuration.ConfigBinder.configBinder; import static io.trino.plugin.jdbc.JdbcModule.bindSessionPropertiesProvider; @@ -176,4 +181,18 @@ private static Configuration readConfig(PhoenixConfig config) } return result; } + + @Singleton + @ForRecordCursor + @Provides + public ExecutorService createRecordCursorExecutor() + { + return newDirectExecutorService(); + } + + @PreDestroy + public void shutdownRecordCursorExecutor(@ForRecordCursor ExecutorService executor) + { + executor.shutdownNow(); + } } diff --git a/plugin/trino-phoenix/src/main/java/io/trino/plugin/phoenix/PhoenixConnectorFactory.java b/plugin/trino-phoenix/src/main/java/io/trino/plugin/phoenix/PhoenixConnectorFactory.java index 99a7cdbf094e..3b08dcdb84db 100644 --- a/plugin/trino-phoenix/src/main/java/io/trino/plugin/phoenix/PhoenixConnectorFactory.java +++ b/plugin/trino-phoenix/src/main/java/io/trino/plugin/phoenix/PhoenixConnectorFactory.java @@ -16,6 +16,7 @@ import com.google.inject.Injector; import io.airlift.bootstrap.Bootstrap; import io.airlift.json.JsonModule; +import io.trino.plugin.base.CatalogName; import io.trino.plugin.base.jmx.MBeanServerModule; import io.trino.spi.classloader.ThreadContextClassLoader; import io.trino.spi.connector.Connector; @@ -63,6 +64,7 @@ public Connector create(String catalogName, Map requiredConfig, new MBeanModule(), new PhoenixClientModule(catalogName), binder -> { + binder.bind(CatalogName.class).toInstance(new CatalogName(catalogName)); binder.bind(ClassLoader.class).toInstance(PhoenixConnectorFactory.class.getClassLoader()); binder.bind(TypeManager.class).toInstance(context.getTypeManager()); }); diff --git a/plugin/trino-phoenix/src/main/java/io/trino/plugin/phoenix/PhoenixMetadata.java b/plugin/trino-phoenix/src/main/java/io/trino/plugin/phoenix/PhoenixMetadata.java index 18bf5fcea8a5..cd09e203a156 100644 --- a/plugin/trino-phoenix/src/main/java/io/trino/plugin/phoenix/PhoenixMetadata.java +++ b/plugin/trino-phoenix/src/main/java/io/trino/plugin/phoenix/PhoenixMetadata.java @@ -30,6 +30,7 @@ import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.ConnectorTableHandle; import io.trino.spi.connector.ConnectorTableMetadata; +import io.trino.spi.connector.ConnectorTableSchema; import io.trino.spi.connector.SchemaTableName; import io.trino.spi.security.TrinoPrincipal; import io.trino.spi.statistics.ComputedStatistics; @@ -83,19 +84,32 @@ public JdbcTableHandle getTableHandle(ConnectorSession session, SchemaTableName } @Override - public ConnectorTableMetadata getTableMetadata(ConnectorSession session, ConnectorTableHandle table) + public ConnectorTableSchema getTableSchema(ConnectorSession session, ConnectorTableHandle table) { - return getTableMetadata(session, table, false); + JdbcTableHandle handle = (JdbcTableHandle) table; + return new ConnectorTableSchema( + getSchemaTableName(handle), + getColumnMetadata(session, handle).stream() + .map(ColumnMetadata::getColumnSchema) + .collect(toImmutableList())); } - private ConnectorTableMetadata getTableMetadata(ConnectorSession session, ConnectorTableHandle table, boolean rowkeyRequired) + @Override + public ConnectorTableMetadata getTableMetadata(ConnectorSession session, ConnectorTableHandle table) { JdbcTableHandle handle = (JdbcTableHandle) table; - List columnMetadata = phoenixClient.getColumns(session, handle).stream() - .filter(column -> rowkeyRequired || !ROWKEY.equalsIgnoreCase(column.getColumnName())) + return new ConnectorTableMetadata( + getSchemaTableName(handle), + getColumnMetadata(session, handle), + phoenixClient.getTableProperties(session, handle)); + } + + private List getColumnMetadata(ConnectorSession session, JdbcTableHandle handle) + { + return phoenixClient.getColumns(session, handle).stream() + .filter(column -> !ROWKEY.equalsIgnoreCase(column.getColumnName())) .map(JdbcColumnHandle::getColumnMetadata) .collect(toImmutableList()); - return new ConnectorTableMetadata(handle.getRequiredNamedRelation().getSchemaTableName(), columnMetadata, phoenixClient.getTableProperties(session, handle)); } @Override diff --git a/plugin/trino-phoenix/src/main/java/io/trino/plugin/phoenix/PhoenixSplitManager.java b/plugin/trino-phoenix/src/main/java/io/trino/plugin/phoenix/PhoenixSplitManager.java index c20c7bdd71dc..9bd5b2ce6575 100644 --- a/plugin/trino-phoenix/src/main/java/io/trino/plugin/phoenix/PhoenixSplitManager.java +++ b/plugin/trino-phoenix/src/main/java/io/trino/plugin/phoenix/PhoenixSplitManager.java @@ -14,12 +14,9 @@ package io.trino.plugin.phoenix; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import io.airlift.log.Logger; import io.trino.plugin.jdbc.JdbcColumnHandle; import io.trino.plugin.jdbc.JdbcTableHandle; -import io.trino.plugin.jdbc.PreparedQuery; -import io.trino.plugin.jdbc.QueryBuilder; import io.trino.spi.HostAddress; import io.trino.spi.TrinoException; import io.trino.spi.connector.ConnectorSession; @@ -83,17 +80,12 @@ public ConnectorSplitSource getSplits( List columns = tableHandle.getColumns() .map(columnSet -> columnSet.stream().map(JdbcColumnHandle.class::cast).collect(toList())) .orElseGet(() -> phoenixClient.getColumns(session, tableHandle)); - QueryBuilder queryBuilder = new QueryBuilder(phoenixClient); - PreparedQuery preparedQuery = queryBuilder.prepareQuery( + PhoenixPreparedStatement inputQuery = (PhoenixPreparedStatement) phoenixClient.prepareStatement( session, connection, - tableHandle.getRelationHandle(), - Optional.empty(), + tableHandle, columns, - ImmutableMap.of(), - tableHandle.getConstraint(), Optional.empty()); - PhoenixPreparedStatement inputQuery = (PhoenixPreparedStatement) queryBuilder.prepareStatement(session, connection, preparedQuery); List splits = getSplits(inputQuery).stream() .map(PhoenixInputSplit.class::cast) diff --git a/plugin/trino-phoenix/src/test/java/io/trino/plugin/phoenix/TestPhoenixPlugin.java b/plugin/trino-phoenix/src/test/java/io/trino/plugin/phoenix/TestPhoenixPlugin.java new file mode 100644 index 000000000000..bceec9859746 --- /dev/null +++ b/plugin/trino-phoenix/src/test/java/io/trino/plugin/phoenix/TestPhoenixPlugin.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 io.trino.plugin.phoenix; + +import com.google.common.collect.ImmutableMap; +import io.trino.spi.Plugin; +import io.trino.spi.connector.ConnectorFactory; +import io.trino.testing.TestingConnectorContext; +import org.testng.annotations.Test; + +import static com.google.common.collect.Iterables.getOnlyElement; + +public class TestPhoenixPlugin +{ + @Test + public void testCreateConnector() + { + Plugin plugin = new PhoenixPlugin(); + ConnectorFactory factory = getOnlyElement(plugin.getConnectorFactories()); + factory.create("test", ImmutableMap.of("phoenix.connection-url", "jdbc:phoenix:test"), new TestingConnectorContext()) + .shutdown(); + } +} diff --git a/plugin/trino-phoenix5/pom.xml b/plugin/trino-phoenix5/pom.xml index b0473b9625f0..b29e1f7f286d 100644 --- a/plugin/trino-phoenix5/pom.xml +++ b/plugin/trino-phoenix5/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml @@ -74,6 +74,11 @@ guice + + javax.annotation + javax.annotation-api + + javax.inject javax.inject diff --git a/plugin/trino-phoenix5/src/main/java/io/trino/plugin/phoenix5/PhoenixClient.java b/plugin/trino-phoenix5/src/main/java/io/trino/plugin/phoenix5/PhoenixClient.java index e9154e25e706..1e34b86f4222 100644 --- a/plugin/trino-phoenix5/src/main/java/io/trino/plugin/phoenix5/PhoenixClient.java +++ b/plugin/trino-phoenix5/src/main/java/io/trino/plugin/phoenix5/PhoenixClient.java @@ -17,10 +17,12 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import io.trino.plugin.jdbc.BaseJdbcClient; +import io.trino.plugin.jdbc.BaseJdbcClient.TopNFunction; import io.trino.plugin.jdbc.ColumnMapping; import io.trino.plugin.jdbc.ConnectionFactory; import io.trino.plugin.jdbc.JdbcColumnHandle; import io.trino.plugin.jdbc.JdbcOutputTableHandle; +import io.trino.plugin.jdbc.JdbcSortItem; import io.trino.plugin.jdbc.JdbcSplit; import io.trino.plugin.jdbc.JdbcTableHandle; import io.trino.plugin.jdbc.JdbcTypeHandle; @@ -185,6 +187,7 @@ public class PhoenixClient extends BaseJdbcClient { private static final String ROWKEY = "ROWKEY"; + private static final long MAX_TOPN_LIMIT = 2000000; private final Configuration configuration; @@ -244,15 +247,12 @@ protected Collection listSchemas(Connection connection) public PreparedStatement buildSql(ConnectorSession session, Connection connection, JdbcSplit split, JdbcTableHandle table, List columnHandles) throws SQLException { - PreparedQuery preparedQuery = prepareQuery( + PreparedStatement query = prepareStatement( session, connection, table, - Optional.empty(), columnHandles, - ImmutableMap.of(), Optional.of(split)); - PreparedStatement query = new QueryBuilder(this).prepareStatement(session, connection, preparedQuery); QueryPlan queryPlan = getQueryPlan((PhoenixPreparedStatement) query); ResultSet resultSet = getResultSet(((PhoenixSplit) split).getPhoenixInputSplit(), queryPlan); return new DelegatePreparedStatement(query) @@ -265,6 +265,52 @@ public ResultSet executeQuery() }; } + public PreparedStatement prepareStatement( + ConnectorSession session, + Connection connection, + JdbcTableHandle table, + List columns, + Optional split) + throws SQLException + { + PreparedQuery preparedQuery = prepareQuery( + session, + connection, + table, + Optional.empty(), + columns, + ImmutableMap.of(), + split); + return new QueryBuilder(this).prepareStatement(session, connection, preparedQuery); + } + + @Override + public boolean supportsTopN(ConnectorSession session, JdbcTableHandle handle, List sortOrder) + { + return true; + } + + @Override + protected Optional topNFunction() + { + return Optional.of((query, sortItems, limit) -> { + // TODO: Remove when this is fixed in Phoenix. + // Phoenix severely over-estimates the memory + // required to execute a topN query. + // https://issues.apache.org/jira/browse/PHOENIX-6436 + if (limit > MAX_TOPN_LIMIT) { + return query; + } + return TopNFunction.sqlStandard(this::quoted).apply(query, sortItems, limit); + }); + } + + @Override + public boolean isTopNLimitGuaranteed(ConnectorSession session) + { + return false; + } + @Override protected Optional> limitFunction() { @@ -274,7 +320,6 @@ protected Optional> limitFunction() @Override public boolean isLimitGuaranteed(ConnectorSession session) { - // Note that limit exceeding Integer.MAX_VALUE gets completely ignored. return false; } diff --git a/plugin/trino-phoenix5/src/main/java/io/trino/plugin/phoenix5/PhoenixClientModule.java b/plugin/trino-phoenix5/src/main/java/io/trino/plugin/phoenix5/PhoenixClientModule.java index ab8c863b9511..eac2f05ad92e 100644 --- a/plugin/trino-phoenix5/src/main/java/io/trino/plugin/phoenix5/PhoenixClientModule.java +++ b/plugin/trino-phoenix5/src/main/java/io/trino/plugin/phoenix5/PhoenixClientModule.java @@ -27,6 +27,7 @@ import io.trino.plugin.base.util.LoggingInvocationHandler; import io.trino.plugin.jdbc.ConnectionFactory; import io.trino.plugin.jdbc.DriverConnectionFactory; +import io.trino.plugin.jdbc.ForRecordCursor; import io.trino.plugin.jdbc.ForwardingJdbcClient; import io.trino.plugin.jdbc.JdbcClient; import io.trino.plugin.jdbc.JdbcMetadataConfig; @@ -49,13 +50,17 @@ import org.apache.phoenix.jdbc.PhoenixDriver; import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver; +import javax.annotation.PreDestroy; + import java.sql.DriverManager; import java.sql.SQLException; import java.util.Map; import java.util.Properties; +import java.util.concurrent.ExecutorService; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.reflect.Reflection.newProxy; +import static com.google.common.util.concurrent.MoreExecutors.newDirectExecutorService; import static com.google.inject.multibindings.OptionalBinder.newOptionalBinder; import static io.airlift.configuration.ConfigBinder.configBinder; import static io.trino.plugin.jdbc.JdbcModule.bindSessionPropertiesProvider; @@ -176,4 +181,18 @@ private static Configuration readConfig(PhoenixConfig config) } return result; } + + @Singleton + @ForRecordCursor + @Provides + public ExecutorService createRecordCursorExecutor() + { + return newDirectExecutorService(); + } + + @PreDestroy + public void shutdownRecordCursorExecutor(@ForRecordCursor ExecutorService executor) + { + executor.shutdownNow(); + } } diff --git a/plugin/trino-phoenix5/src/main/java/io/trino/plugin/phoenix5/PhoenixConnectorFactory.java b/plugin/trino-phoenix5/src/main/java/io/trino/plugin/phoenix5/PhoenixConnectorFactory.java index 2fc40d6fb1b1..9298716f24da 100644 --- a/plugin/trino-phoenix5/src/main/java/io/trino/plugin/phoenix5/PhoenixConnectorFactory.java +++ b/plugin/trino-phoenix5/src/main/java/io/trino/plugin/phoenix5/PhoenixConnectorFactory.java @@ -16,6 +16,7 @@ import com.google.inject.Injector; import io.airlift.bootstrap.Bootstrap; import io.airlift.json.JsonModule; +import io.trino.plugin.base.CatalogName; import io.trino.plugin.base.jmx.MBeanServerModule; import io.trino.spi.classloader.ThreadContextClassLoader; import io.trino.spi.connector.Connector; @@ -63,6 +64,7 @@ public Connector create(String catalogName, Map requiredConfig, new MBeanModule(), new PhoenixClientModule(catalogName), binder -> { + binder.bind(CatalogName.class).toInstance(new CatalogName(catalogName)); binder.bind(ClassLoader.class).toInstance(PhoenixConnectorFactory.class.getClassLoader()); binder.bind(TypeManager.class).toInstance(context.getTypeManager()); }); diff --git a/plugin/trino-phoenix5/src/main/java/io/trino/plugin/phoenix5/PhoenixMetadata.java b/plugin/trino-phoenix5/src/main/java/io/trino/plugin/phoenix5/PhoenixMetadata.java index 856531a68c0a..6087d455db96 100644 --- a/plugin/trino-phoenix5/src/main/java/io/trino/plugin/phoenix5/PhoenixMetadata.java +++ b/plugin/trino-phoenix5/src/main/java/io/trino/plugin/phoenix5/PhoenixMetadata.java @@ -30,6 +30,7 @@ import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.ConnectorTableHandle; import io.trino.spi.connector.ConnectorTableMetadata; +import io.trino.spi.connector.ConnectorTableSchema; import io.trino.spi.connector.SchemaTableName; import io.trino.spi.security.TrinoPrincipal; import io.trino.spi.statistics.ComputedStatistics; @@ -83,19 +84,32 @@ public JdbcTableHandle getTableHandle(ConnectorSession session, SchemaTableName } @Override - public ConnectorTableMetadata getTableMetadata(ConnectorSession session, ConnectorTableHandle table) + public ConnectorTableSchema getTableSchema(ConnectorSession session, ConnectorTableHandle table) { - return getTableMetadata(session, table, false); + JdbcTableHandle handle = (JdbcTableHandle) table; + return new ConnectorTableSchema( + getSchemaTableName(handle), + getColumnMetadata(session, handle).stream() + .map(ColumnMetadata::getColumnSchema) + .collect(toImmutableList())); } - private ConnectorTableMetadata getTableMetadata(ConnectorSession session, ConnectorTableHandle table, boolean rowkeyRequired) + @Override + public ConnectorTableMetadata getTableMetadata(ConnectorSession session, ConnectorTableHandle table) { JdbcTableHandle handle = (JdbcTableHandle) table; - List columnMetadata = phoenixClient.getColumns(session, handle).stream() - .filter(column -> rowkeyRequired || !ROWKEY.equalsIgnoreCase(column.getColumnName())) + return new ConnectorTableMetadata( + getSchemaTableName(handle), + getColumnMetadata(session, handle), + phoenixClient.getTableProperties(session, handle)); + } + + private List getColumnMetadata(ConnectorSession session, JdbcTableHandle handle) + { + return phoenixClient.getColumns(session, handle).stream() + .filter(column -> !ROWKEY.equalsIgnoreCase(column.getColumnName())) .map(JdbcColumnHandle::getColumnMetadata) .collect(toImmutableList()); - return new ConnectorTableMetadata(handle.getRequiredNamedRelation().getSchemaTableName(), columnMetadata, phoenixClient.getTableProperties(session, handle)); } @Override diff --git a/plugin/trino-phoenix5/src/main/java/io/trino/plugin/phoenix5/PhoenixSplitManager.java b/plugin/trino-phoenix5/src/main/java/io/trino/plugin/phoenix5/PhoenixSplitManager.java index ea04914b3d2b..878bf46ae273 100644 --- a/plugin/trino-phoenix5/src/main/java/io/trino/plugin/phoenix5/PhoenixSplitManager.java +++ b/plugin/trino-phoenix5/src/main/java/io/trino/plugin/phoenix5/PhoenixSplitManager.java @@ -14,12 +14,9 @@ package io.trino.plugin.phoenix5; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import io.airlift.log.Logger; import io.trino.plugin.jdbc.JdbcColumnHandle; import io.trino.plugin.jdbc.JdbcTableHandle; -import io.trino.plugin.jdbc.PreparedQuery; -import io.trino.plugin.jdbc.QueryBuilder; import io.trino.spi.HostAddress; import io.trino.spi.TrinoException; import io.trino.spi.connector.ConnectorSession; @@ -83,17 +80,12 @@ public ConnectorSplitSource getSplits( List columns = tableHandle.getColumns() .map(columnSet -> columnSet.stream().map(JdbcColumnHandle.class::cast).collect(toList())) .orElseGet(() -> phoenixClient.getColumns(session, tableHandle)); - QueryBuilder queryBuilder = new QueryBuilder(phoenixClient); - PreparedQuery preparedQuery = queryBuilder.prepareQuery( + PhoenixPreparedStatement inputQuery = (PhoenixPreparedStatement) phoenixClient.prepareStatement( session, connection, - tableHandle.getRelationHandle(), - Optional.empty(), + tableHandle, columns, - ImmutableMap.of(), - tableHandle.getConstraint(), Optional.empty()); - PhoenixPreparedStatement inputQuery = (PhoenixPreparedStatement) queryBuilder.prepareStatement(session, connection, preparedQuery); List splits = getSplits(inputQuery).stream() .map(PhoenixInputSplit.class::cast) diff --git a/plugin/trino-phoenix5/src/test/java/io/trino/plugin/phoenix5/TestPhoenixPlugin.java b/plugin/trino-phoenix5/src/test/java/io/trino/plugin/phoenix5/TestPhoenixPlugin.java new file mode 100644 index 000000000000..a54f168d1131 --- /dev/null +++ b/plugin/trino-phoenix5/src/test/java/io/trino/plugin/phoenix5/TestPhoenixPlugin.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 io.trino.plugin.phoenix5; + +import com.google.common.collect.ImmutableMap; +import io.trino.spi.Plugin; +import io.trino.spi.connector.ConnectorFactory; +import io.trino.testing.TestingConnectorContext; +import org.testng.annotations.Test; + +import static com.google.common.collect.Iterables.getOnlyElement; + +public class TestPhoenixPlugin +{ + @Test + public void testCreateConnector() + { + Plugin plugin = new PhoenixPlugin(); + ConnectorFactory factory = getOnlyElement(plugin.getConnectorFactories()); + factory.create("test", ImmutableMap.of("phoenix.connection-url", "jdbc:phoenix:test"), new TestingConnectorContext()) + .shutdown(); + } +} diff --git a/plugin/trino-pinot/pom.xml b/plugin/trino-pinot/pom.xml index c5d55810367e..2a1c702b2146 100755 --- a/plugin/trino-pinot/pom.xml +++ b/plugin/trino-pinot/pom.xml @@ -4,7 +4,7 @@ trino-root io.trino - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-postgresql/pom.xml b/plugin/trino-postgresql/pom.xml index 69ab20a9c10f..f376a3ee86cd 100644 --- a/plugin/trino-postgresql/pom.xml +++ b/plugin/trino-postgresql/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml @@ -33,6 +33,11 @@ trino-plugin-toolkit + + io.airlift + concurrent + + io.airlift configuration @@ -43,6 +48,11 @@ log + + io.airlift + units + + com.google.guava guava @@ -68,6 +78,11 @@ joda-time + + net.jodah + failsafe + + org.postgresql postgresql @@ -186,6 +201,12 @@ test + + org.testcontainers + testcontainers + test + + org.testng testng diff --git a/plugin/trino-postgresql/src/main/java/io/trino/plugin/postgresql/PostgreSqlClient.java b/plugin/trino-postgresql/src/main/java/io/trino/plugin/postgresql/PostgreSqlClient.java index 32730bebc084..20f2668980dd 100644 --- a/plugin/trino-postgresql/src/main/java/io/trino/plugin/postgresql/PostgreSqlClient.java +++ b/plugin/trino-postgresql/src/main/java/io/trino/plugin/postgresql/PostgreSqlClient.java @@ -202,6 +202,7 @@ import static java.math.RoundingMode.UNNECESSARY; import static java.sql.DatabaseMetaData.columnNoNulls; import static java.util.Objects.requireNonNull; +import static java.util.stream.Collectors.joining; public class PostgreSqlClient extends BaseJdbcClient @@ -686,10 +687,9 @@ public boolean supportsTopN(ConnectorSession session, JdbcTableHandle handle, Li for (JdbcSortItem sortItem : sortOrder) { Type sortItemType = sortItem.getColumn().getColumnType(); if (sortItemType instanceof CharType || sortItemType instanceof VarcharType) { - // PostgreSQL by default orders lowercase letters before uppercase, which is different from Trino - // NOTE: VarcharType also includes PostgreSQL enums - // TODO We could still push the sort down if we could inject a PostgreSQL-specific syntax for selecting a collation for given comparison. - return false; + if (!isCollatable(sortItem.getColumn())) { + return false; + } } } return true; @@ -698,7 +698,33 @@ public boolean supportsTopN(ConnectorSession session, JdbcTableHandle handle, Li @Override protected Optional topNFunction() { - return Optional.of(TopNFunction.sqlStandard(this::quoted)); + return Optional.of((query, sortItems, limit) -> { + String orderBy = sortItems.stream() + .map(sortItem -> { + String ordering = sortItem.getSortOrder().isAscending() ? "ASC" : "DESC"; + String nullsHandling = sortItem.getSortOrder().isNullsFirst() ? "NULLS FIRST" : "NULLS LAST"; + String collation = ""; + if (isCollatable(sortItem.getColumn())) { + collation = "COLLATE \"C\""; + } + return format("%s %s %s %s", quoted(sortItem.getColumn().getColumnName()), collation, ordering, nullsHandling); + }) + .collect(joining(", ")); + return format("%s ORDER BY %s LIMIT %d", query, orderBy, limit); + }); + } + + private boolean isCollatable(JdbcColumnHandle column) + { + if (column.getColumnType() instanceof CharType || column.getColumnType() instanceof VarcharType) { + String jdbcTypeName = column.getJdbcTypeHandle().getJdbcTypeName() + .orElseThrow(() -> new TrinoException(JDBC_ERROR, "Type name is missing: " + column.getJdbcTypeHandle())); + // Only char (internally named bpchar)/varchar/text are the built-in collatable types + return "bpchar".equals(jdbcTypeName) || "varchar".equals(jdbcTypeName) || "text".equals(jdbcTypeName); + } + + // non-textual types don't have the concept of collation + return false; } @Override diff --git a/plugin/trino-postgresql/src/main/java/io/trino/plugin/postgresql/PostgreSqlClientModule.java b/plugin/trino-postgresql/src/main/java/io/trino/plugin/postgresql/PostgreSqlClientModule.java index a965a6b5bfbf..17e21438f193 100644 --- a/plugin/trino-postgresql/src/main/java/io/trino/plugin/postgresql/PostgreSqlClientModule.java +++ b/plugin/trino-postgresql/src/main/java/io/trino/plugin/postgresql/PostgreSqlClientModule.java @@ -14,16 +14,17 @@ package io.trino.plugin.postgresql; import com.google.inject.Binder; -import com.google.inject.Module; import com.google.inject.Provides; import com.google.inject.Scopes; import com.google.inject.Singleton; +import io.airlift.configuration.AbstractConfigurationAwareModule; import io.trino.plugin.jdbc.BaseJdbcConfig; import io.trino.plugin.jdbc.ConnectionFactory; import io.trino.plugin.jdbc.DecimalModule; import io.trino.plugin.jdbc.DriverConnectionFactory; import io.trino.plugin.jdbc.ForBaseJdbc; import io.trino.plugin.jdbc.JdbcClient; +import io.trino.plugin.jdbc.RemoteQueryCancellationModule; import io.trino.plugin.jdbc.credential.CredentialProvider; import org.postgresql.Driver; @@ -31,15 +32,16 @@ import static io.trino.plugin.jdbc.JdbcModule.bindSessionPropertiesProvider; public class PostgreSqlClientModule - implements Module + extends AbstractConfigurationAwareModule { @Override - public void configure(Binder binder) + public void setup(Binder binder) { binder.bind(JdbcClient.class).annotatedWith(ForBaseJdbc.class).to(PostgreSqlClient.class).in(Scopes.SINGLETON); configBinder(binder).bindConfig(PostgreSqlConfig.class); bindSessionPropertiesProvider(binder, PostgreSqlSessionProperties.class); - binder.install(new DecimalModule()); + install(new DecimalModule()); + install(new RemoteQueryCancellationModule()); } @Provides diff --git a/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlConnectorTest.java b/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlConnectorTest.java index dff7e80ba6f8..a7358928e8d2 100644 --- a/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlConnectorTest.java +++ b/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlConnectorTest.java @@ -14,8 +14,10 @@ package io.trino.plugin.postgresql; import com.google.common.collect.ImmutableList; +import io.airlift.units.Duration; import io.trino.Session; import io.trino.plugin.jdbc.BaseJdbcConnectorTest; +import io.trino.plugin.jdbc.RemoteDatabaseEvent; import io.trino.sql.planner.plan.AggregationNode; import io.trino.sql.planner.plan.ExchangeNode; import io.trino.sql.planner.plan.FilterNode; @@ -29,6 +31,7 @@ import io.trino.testing.sql.JdbcSqlExecutor; import io.trino.testing.sql.SqlExecutor; import io.trino.testing.sql.TestTable; +import io.trino.testing.sql.TestView; import org.intellij.lang.annotations.Language; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; @@ -46,7 +49,9 @@ import static io.trino.sql.planner.assertions.PlanMatchPattern.anyTree; import static io.trino.sql.planner.assertions.PlanMatchPattern.node; import static io.trino.testing.sql.TestTable.randomTableSuffix; +import static java.lang.Math.round; import static java.lang.String.format; +import static java.util.concurrent.TimeUnit.SECONDS; import static java.util.stream.Collectors.joining; import static java.util.stream.IntStream.range; import static org.assertj.core.api.Assertions.assertThat; @@ -82,6 +87,7 @@ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior) return false; case SUPPORTS_TOPN_PUSHDOWN: + case SUPPORTS_TOPN_PUSHDOWN_WITH_VARCHAR: return true; case SUPPORTS_JOIN_PUSHDOWN: @@ -97,6 +103,9 @@ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior) case SUPPORTS_RENAME_TABLE_ACROSS_SCHEMAS: return false; + case SUPPORTS_CANCELLATION: + return true; + default: return super.hasBehavior(connectorBehavior); } @@ -763,7 +772,7 @@ public void testLimitPushdown() // with TopN over numeric column assertThat(query("SELECT * FROM (SELECT regionkey FROM nation ORDER BY nationkey ASC LIMIT 10) LIMIT 5")).isFullyPushedDown(); // with TopN over varchar column - assertThat(query("SELECT * FROM (SELECT regionkey FROM nation ORDER BY name ASC LIMIT 10) LIMIT 5")).isNotFullyPushedDown(TopNNode.class); + assertThat(query("SELECT * FROM (SELECT regionkey FROM nation ORDER BY name ASC LIMIT 10) LIMIT 5")).isFullyPushedDown(); // LIMIT with JOIN assertThat(query(joinPushdownEnabled(getSession()), "" + @@ -880,4 +889,17 @@ private void execute(String sql) statement.execute(sql); } } + + @Override + protected List getRemoteDatabaseEvents() + { + return postgreSqlServer.getRemoteDatabaseEvents(); + } + + @Override + protected TestView createSleepingView(Duration minimalQueryDuration) + { + long secondsToSleep = round(minimalQueryDuration.convertTo(SECONDS).getValue() + 1); + return new TestView(onRemoteDatabase(), format("SELECT 1 FROM pg_sleep(%d)", secondsToSleep)); + } } diff --git a/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlTypeMapping.java b/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlTypeMapping.java index 156606db9a1c..b3b098556e20 100644 --- a/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlTypeMapping.java +++ b/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlTypeMapping.java @@ -216,6 +216,7 @@ public void testDouble() SqlDataTypeTest.create() .addRoundTrip("double precision", "NULL", DOUBLE, "CAST(NULL AS double)") .addRoundTrip("double precision", "1.0E100", DOUBLE, "1.0E100") + .addRoundTrip("double precision", "123.456E10", DOUBLE, "123.456E10") .addRoundTrip("double precision", "'NaN'::double precision", DOUBLE, "nan()") .addRoundTrip("double precision", "'+Infinity'::double precision", DOUBLE, "+infinity()") .addRoundTrip("double precision", "'-Infinity'::double precision", DOUBLE, "-infinity()") @@ -224,6 +225,7 @@ public void testDouble() SqlDataTypeTest.create() .addRoundTrip("double", "NULL", DOUBLE, "CAST(NULL AS double)") .addRoundTrip("double", "1.0E100", DOUBLE, "1.0E100") + .addRoundTrip("double", "123.456E10", DOUBLE, "123.456E10") .addRoundTrip("double", "nan()", DOUBLE, "nan()") .addRoundTrip("double", "+infinity()", DOUBLE, "+infinity()") .addRoundTrip("double", "-infinity()", DOUBLE, "-infinity()") diff --git a/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestTestingPostgreSqlServer.java b/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestTestingPostgreSqlServer.java new file mode 100644 index 000000000000..af924e64030d --- /dev/null +++ b/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestTestingPostgreSqlServer.java @@ -0,0 +1,102 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.trino.plugin.postgresql; + +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Sets; +import io.trino.plugin.jdbc.RemoteDatabaseEvent; +import net.jodah.failsafe.function.CheckedRunnable; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.List; +import java.util.Set; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; + +import static io.airlift.concurrent.Threads.daemonThreadsNamed; +import static io.airlift.testing.Closeables.closeAll; +import static io.trino.plugin.jdbc.RemoteDatabaseEvent.Status.CANCELLED; +import static io.trino.plugin.jdbc.RemoteDatabaseEvent.Status.RUNNING; +import static java.util.concurrent.Executors.newCachedThreadPool; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +public class TestTestingPostgreSqlServer +{ + private final ExecutorService threadPool = newCachedThreadPool(daemonThreadsNamed("TestTestingPostgreSqlServer-%d")); + + private TestingPostgreSqlServer postgreSqlServer; + + @BeforeClass + public void setUp() + { + postgreSqlServer = new TestingPostgreSqlServer(); + } + + @AfterClass(alwaysRun = true) + public void tearDown() + throws Exception + { + closeAll( + postgreSqlServer, + () -> threadPool.shutdownNow()); + } + + @Test + public void testCapturingSuccessfulStatement() + throws Throwable + { + String sql = "SELECT 1"; + Set remoteDatabaseEvents = captureRemoteEventsDuring(() -> postgreSqlServer.execute(sql)); + assertThat(remoteDatabaseEvents).contains(new RemoteDatabaseEvent(sql, RUNNING)); + } + + @Test(timeOut = 60_000) + public void testCapturingCancelledStatement() + throws Throwable + { + String sql = "SELECT pg_sleep(60)"; + Set remoteDatabaseEvents = captureRemoteEventsDuring(() -> { + try (Connection connection = DriverManager.getConnection(postgreSqlServer.getJdbcUrl(), postgreSqlServer.getProperties()); + Statement statement = connection.createStatement()) { + Future executeFuture = threadPool.submit(() -> statement.execute(sql)); + Thread.sleep(5000); + statement.cancel(); + assertThatThrownBy(() -> executeFuture.get()) + .hasRootCauseInstanceOf(SQLException.class) + .hasRootCauseMessage("ERROR: canceling statement due to user request"); + } + catch (SQLException e) { + throw new RuntimeException(e); + } + }); + assertThat(remoteDatabaseEvents).contains(new RemoteDatabaseEvent(sql, RUNNING)); + assertThat(remoteDatabaseEvents).contains(new RemoteDatabaseEvent(sql, CANCELLED)); + } + + private Set captureRemoteEventsDuring(CheckedRunnable runnable) + throws Throwable + { + List before = postgreSqlServer.getRemoteDatabaseEvents(); + runnable.run(); + List after = postgreSqlServer.getRemoteDatabaseEvents(); + return Sets.difference(ImmutableSet.copyOf(after), ImmutableSet.copyOf(before)); + } +} diff --git a/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestingPostgreSqlServer.java b/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestingPostgreSqlServer.java index ae1c1beef874..f35fd3460bc6 100644 --- a/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestingPostgreSqlServer.java +++ b/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestingPostgreSqlServer.java @@ -13,6 +13,8 @@ */ package io.trino.plugin.postgresql; +import com.google.common.collect.ImmutableList; +import io.trino.plugin.jdbc.RemoteDatabaseEvent; import org.testcontainers.containers.PostgreSQLContainer; import java.io.Closeable; @@ -20,9 +22,16 @@ import java.sql.DriverManager; import java.sql.SQLException; import java.sql.Statement; +import java.util.Iterator; +import java.util.List; import java.util.Properties; +import java.util.stream.Stream; +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.trino.plugin.jdbc.RemoteDatabaseEvent.Status.CANCELLED; +import static io.trino.plugin.jdbc.RemoteDatabaseEvent.Status.RUNNING; import static java.lang.String.format; +import static java.util.function.Predicate.not; import static org.testcontainers.containers.PostgreSQLContainer.POSTGRESQL_PORT; public class TestingPostgreSqlServer @@ -32,6 +41,10 @@ public class TestingPostgreSqlServer private static final String PASSWORD = "test"; private static final String DATABASE = "tpch"; + private static final String LOG_RUNNING_STATEMENT_PREFIX = "LOG: execute : "; + private static final String LOG_CANCELLATION_EVENT = "ERROR: canceling statement due to user request"; + private static final String LOG_CANCELLED_STATEMENT_PREFIX = "STATEMENT: "; + private final PostgreSQLContainer dockerContainer; public TestingPostgreSqlServer() @@ -40,7 +53,8 @@ public TestingPostgreSqlServer() dockerContainer = new PostgreSQLContainer<>("postgres:9.5") .withDatabaseName(DATABASE) .withUsername(USER) - .withPassword(PASSWORD); + .withPassword(PASSWORD) + .withCommand("postgres", "-c", "log_destination=stderr", "-c", "log_statement=all"); dockerContainer.start(); execute("CREATE SCHEMA tpch"); @@ -62,6 +76,35 @@ private static void execute(String url, Properties properties, String sql) } } + protected List getRemoteDatabaseEvents() + { + List logs = getLogs(); + Iterator logsIterator = logs.iterator(); + ImmutableList.Builder events = ImmutableList.builder(); + while (logsIterator.hasNext()) { + String logLine = logsIterator.next(); + if (logLine.startsWith(LOG_RUNNING_STATEMENT_PREFIX)) { + events.add(new RemoteDatabaseEvent(logLine.substring(LOG_RUNNING_STATEMENT_PREFIX.length()), RUNNING)); + } + if (logLine.equals(LOG_CANCELLATION_EVENT)) { + // next line must be present + String cancelledStatementLogLine = logsIterator.next(); + if (cancelledStatementLogLine.startsWith(LOG_CANCELLED_STATEMENT_PREFIX)) { + events.add(new RemoteDatabaseEvent(cancelledStatementLogLine.substring(LOG_CANCELLED_STATEMENT_PREFIX.length()), CANCELLED)); + } + } + // ignore unsupported log lines + } + return events.build(); + } + + private List getLogs() + { + return Stream.of(dockerContainer.getLogs().split("\n")) + .filter(not(String::isBlank)) + .collect(toImmutableList()); + } + public String getUser() { return USER; diff --git a/plugin/trino-prometheus/pom.xml b/plugin/trino-prometheus/pom.xml index a5d58529a11c..8a19dd06cb9a 100644 --- a/plugin/trino-prometheus/pom.xml +++ b/plugin/trino-prometheus/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-prometheus/src/test/java/io/trino/plugin/prometheus/PrometheusQueryRunner.java b/plugin/trino-prometheus/src/test/java/io/trino/plugin/prometheus/PrometheusQueryRunner.java index 9afd85d3a903..d324edf04467 100644 --- a/plugin/trino-prometheus/src/test/java/io/trino/plugin/prometheus/PrometheusQueryRunner.java +++ b/plugin/trino-prometheus/src/test/java/io/trino/plugin/prometheus/PrometheusQueryRunner.java @@ -30,6 +30,8 @@ import static io.trino.metadata.MetadataManager.createTestMetadataManager; import static io.trino.plugin.prometheus.MetadataUtil.METRIC_CODEC; import static io.trino.testing.TestingSession.testSessionBuilder; +import static java.util.concurrent.TimeUnit.DAYS; +import static java.util.concurrent.TimeUnit.SECONDS; public final class PrometheusQueryRunner { @@ -69,9 +71,9 @@ public static PrometheusClient createPrometheusClient(PrometheusServer server) { PrometheusConnectorConfig config = new PrometheusConnectorConfig(); config.setPrometheusURI(server.getUri()); - config.setQueryChunkSizeDuration(Duration.valueOf("1d")); - config.setMaxQueryRangeDuration(Duration.valueOf("21d")); - config.setCacheDuration(Duration.valueOf("30s")); + config.setQueryChunkSizeDuration(new Duration(1, DAYS)); + config.setMaxQueryRangeDuration(new Duration(21, DAYS)); + config.setCacheDuration(new Duration(30, SECONDS)); return new PrometheusClient(config, METRIC_CODEC, TYPE_MANAGER); } diff --git a/plugin/trino-prometheus/src/test/java/io/trino/plugin/prometheus/TestPrometheusConnectorConfig.java b/plugin/trino-prometheus/src/test/java/io/trino/plugin/prometheus/TestPrometheusConnectorConfig.java index 7d0e9bb0f217..cbadd387426f 100644 --- a/plugin/trino-prometheus/src/test/java/io/trino/plugin/prometheus/TestPrometheusConnectorConfig.java +++ b/plugin/trino-prometheus/src/test/java/io/trino/plugin/prometheus/TestPrometheusConnectorConfig.java @@ -26,6 +26,8 @@ import static io.airlift.configuration.testing.ConfigAssertions.assertFullMapping; import static io.airlift.configuration.testing.ConfigAssertions.assertRecordedDefaults; import static io.airlift.configuration.testing.ConfigAssertions.recordDefaults; +import static java.util.concurrent.TimeUnit.DAYS; +import static java.util.concurrent.TimeUnit.SECONDS; import static org.assertj.core.api.Assertions.assertThatThrownBy; public class TestPrometheusConnectorConfig @@ -36,9 +38,9 @@ public void testDefaults() { assertRecordedDefaults(recordDefaults(PrometheusConnectorConfig.class) .setPrometheusURI(new URI("http://localhost:9090")) - .setQueryChunkSizeDuration(Duration.valueOf("1d")) - .setMaxQueryRangeDuration(Duration.valueOf("21d")) - .setCacheDuration(Duration.valueOf("30s")) + .setQueryChunkSizeDuration(new Duration(1, DAYS)) + .setMaxQueryRangeDuration(new Duration(21, DAYS)) + .setCacheDuration(new Duration(30, SECONDS)) .setBearerTokenFile(null)); } @@ -56,9 +58,9 @@ public void testExplicitPropertyMappings() URI uri = URI.create("file://test.json"); PrometheusConnectorConfig expected = new PrometheusConnectorConfig(); expected.setPrometheusURI(uri); - expected.setQueryChunkSizeDuration(Duration.valueOf("365d")); - expected.setMaxQueryRangeDuration(Duration.valueOf("1095d")); - expected.setCacheDuration(Duration.valueOf("60s")); + expected.setQueryChunkSizeDuration(new Duration(365, DAYS)); + expected.setMaxQueryRangeDuration(new Duration(1095, DAYS)); + expected.setCacheDuration(new Duration(60, SECONDS)); expected.setBearerTokenFile(new File("/tmp/bearer_token.txt")); assertFullMapping(properties, expected); @@ -70,9 +72,9 @@ public void testFailOnDurationLessThanQueryChunkConfig() { PrometheusConnectorConfig config = new PrometheusConnectorConfig(); config.setPrometheusURI(new URI("http://doesnotmatter.com")); - config.setQueryChunkSizeDuration(Duration.valueOf("21d")); - config.setMaxQueryRangeDuration(Duration.valueOf("1d")); - config.setCacheDuration(Duration.valueOf("30s")); + config.setQueryChunkSizeDuration(new Duration(21, DAYS)); + config.setMaxQueryRangeDuration(new Duration(1, DAYS)); + config.setCacheDuration(new Duration(30, SECONDS)); assertThatThrownBy(config::checkConfig) .isInstanceOf(ConfigurationException.class) .hasMessageContaining("prometheus.max.query.range.duration must be greater than prometheus.query.chunk.size.duration"); diff --git a/plugin/trino-prometheus/src/test/java/io/trino/plugin/prometheus/TestPrometheusIntegrationSchema.java b/plugin/trino-prometheus/src/test/java/io/trino/plugin/prometheus/TestPrometheusIntegrationSchema.java index 5024e4fbb64d..b3bab7b58936 100644 --- a/plugin/trino-prometheus/src/test/java/io/trino/plugin/prometheus/TestPrometheusIntegrationSchema.java +++ b/plugin/trino-prometheus/src/test/java/io/trino/plugin/prometheus/TestPrometheusIntegrationSchema.java @@ -38,6 +38,8 @@ import static io.trino.spi.type.DoubleType.DOUBLE; import static io.trino.spi.type.VarcharType.createUnboundedVarcharType; import static io.trino.testing.TestingConnectorSession.SESSION; +import static java.util.concurrent.TimeUnit.DAYS; +import static java.util.concurrent.TimeUnit.SECONDS; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; @@ -163,9 +165,9 @@ public void testCorrectNumberOfSplitsCreated() { PrometheusConnectorConfig config = new PrometheusConnectorConfig(); config.setPrometheusURI(server.getUri()); - config.setMaxQueryRangeDuration(Duration.valueOf("21d")); - config.setQueryChunkSizeDuration(Duration.valueOf("1d")); - config.setCacheDuration(Duration.valueOf("30s")); + config.setMaxQueryRangeDuration(new Duration(21, DAYS)); + config.setQueryChunkSizeDuration(new Duration(1, DAYS)); + config.setCacheDuration(new Duration(30, SECONDS)); PrometheusTable table = client.getTable("default", "up"); PrometheusSplitManager splitManager = new PrometheusSplitManager(client, new PrometheusClock(), config); ConnectorSplitSource splits = splitManager.getSplits( diff --git a/plugin/trino-prometheus/src/test/java/io/trino/plugin/prometheus/TestPrometheusSplit.java b/plugin/trino-prometheus/src/test/java/io/trino/plugin/prometheus/TestPrometheusSplit.java index a427cd8675a1..415afd9623fb 100644 --- a/plugin/trino-prometheus/src/test/java/io/trino/plugin/prometheus/TestPrometheusSplit.java +++ b/plugin/trino-prometheus/src/test/java/io/trino/plugin/prometheus/TestPrometheusSplit.java @@ -60,6 +60,8 @@ import static io.trino.spi.type.TimeZoneKey.UTC_KEY; import static java.time.Instant.ofEpochMilli; import static java.time.ZoneOffset.UTC; +import static java.util.concurrent.TimeUnit.DAYS; +import static java.util.concurrent.TimeUnit.SECONDS; import static org.apache.http.client.utils.URLEncodedUtils.parse; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotEquals; @@ -505,9 +507,9 @@ private static PrometheusConnectorConfig getCommonConfig(URI dataUri) { PrometheusConnectorConfig config = new PrometheusConnectorConfig(); config.setPrometheusURI(dataUri); - config.setMaxQueryRangeDuration(io.airlift.units.Duration.valueOf("21d")); - config.setQueryChunkSizeDuration(io.airlift.units.Duration.valueOf("1d")); - config.setCacheDuration(io.airlift.units.Duration.valueOf("30s")); + config.setMaxQueryRangeDuration(new io.airlift.units.Duration(21, DAYS)); + config.setQueryChunkSizeDuration(new io.airlift.units.Duration(1, DAYS)); + config.setCacheDuration(new io.airlift.units.Duration(30, SECONDS)); return config; } } diff --git a/plugin/trino-raptor-legacy/pom.xml b/plugin/trino-raptor-legacy/pom.xml index e5aadfecaf08..7e8dbc0b8b08 100644 --- a/plugin/trino-raptor-legacy/pom.xml +++ b/plugin/trino-raptor-legacy/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-raptor-legacy/src/main/java/io/trino/plugin/raptor/legacy/storage/OrcPageSource.java b/plugin/trino-raptor-legacy/src/main/java/io/trino/plugin/raptor/legacy/storage/RaptorPageSource.java similarity index 99% rename from plugin/trino-raptor-legacy/src/main/java/io/trino/plugin/raptor/legacy/storage/OrcPageSource.java rename to plugin/trino-raptor-legacy/src/main/java/io/trino/plugin/raptor/legacy/storage/RaptorPageSource.java index fb2628e64b9e..f363f4827d32 100644 --- a/plugin/trino-raptor-legacy/src/main/java/io/trino/plugin/raptor/legacy/storage/OrcPageSource.java +++ b/plugin/trino-raptor-legacy/src/main/java/io/trino/plugin/raptor/legacy/storage/RaptorPageSource.java @@ -46,7 +46,7 @@ import static java.lang.Math.toIntExact; import static java.util.Objects.requireNonNull; -public class OrcPageSource +public class RaptorPageSource implements UpdatablePageSource { private final Optional shardRewriter; @@ -61,7 +61,7 @@ public class OrcPageSource private boolean closed; - public OrcPageSource( + public RaptorPageSource( Optional shardRewriter, OrcRecordReader recordReader, List columnAdaptations, diff --git a/plugin/trino-raptor-legacy/src/main/java/io/trino/plugin/raptor/legacy/storage/OrcStorageManager.java b/plugin/trino-raptor-legacy/src/main/java/io/trino/plugin/raptor/legacy/storage/RaptorStorageManager.java similarity index 98% rename from plugin/trino-raptor-legacy/src/main/java/io/trino/plugin/raptor/legacy/storage/OrcStorageManager.java rename to plugin/trino-raptor-legacy/src/main/java/io/trino/plugin/raptor/legacy/storage/RaptorStorageManager.java index ffd2851f2754..ad634e55bce6 100644 --- a/plugin/trino-raptor-legacy/src/main/java/io/trino/plugin/raptor/legacy/storage/OrcStorageManager.java +++ b/plugin/trino-raptor-legacy/src/main/java/io/trino/plugin/raptor/legacy/storage/RaptorStorageManager.java @@ -45,7 +45,7 @@ import io.trino.plugin.raptor.legacy.metadata.ShardInfo; import io.trino.plugin.raptor.legacy.metadata.ShardRecorder; import io.trino.plugin.raptor.legacy.storage.OrcFileRewriter.OrcFileInfo; -import io.trino.plugin.raptor.legacy.storage.OrcPageSource.ColumnAdaptation; +import io.trino.plugin.raptor.legacy.storage.RaptorPageSource.ColumnAdaptation; import io.trino.spi.NodeManager; import io.trino.spi.Page; import io.trino.spi.TrinoException; @@ -131,7 +131,7 @@ import static java.util.stream.Collectors.toList; import static org.joda.time.DateTimeZone.UTC; -public class OrcStorageManager +public class RaptorStorageManager implements StorageManager { private static final JsonCodec SHARD_DELTA_CODEC = jsonCodec(ShardDelta.class); @@ -157,7 +157,7 @@ public class OrcStorageManager private final ExecutorService commitExecutor; @Inject - public OrcStorageManager( + public RaptorStorageManager( NodeManager nodeManager, StorageService storageService, Optional backupStore, @@ -184,7 +184,7 @@ public OrcStorageManager( config.getMinAvailableSpace()); } - public OrcStorageManager( + public RaptorStorageManager( String nodeId, StorageService storageService, Optional backupStore, @@ -280,14 +280,14 @@ public ConnectorPageSource getPageSource( UTC, systemMemoryUsage, INITIAL_BATCH_SIZE, - OrcPageSource::handleException); + RaptorPageSource::handleException); Optional shardRewriter = Optional.empty(); if (transactionId.isPresent()) { shardRewriter = Optional.of(createShardRewriter(transactionId.getAsLong(), bucketNumber, shardUuid)); } - return new OrcPageSource(shardRewriter, recordReader, columnAdaptations, dataSource, systemMemoryUsage); + return new RaptorPageSource(shardRewriter, recordReader, columnAdaptations, dataSource, systemMemoryUsage); } catch (IOException | RuntimeException e) { closeQuietly(dataSource); @@ -319,7 +319,7 @@ public StoragePageSink createStoragePageSink(long transactionId, OptionalInt buc if (checkSpace && storageService.getAvailableBytes() < minAvailableSpace.toBytes()) { throw new TrinoException(RAPTOR_LOCAL_DISK_FULL, "Local disk is full on node " + nodeId); } - return new OrcStoragePageSink(transactionId, columnIds, columnTypes, bucketNumber); + return new RaptorStoragePageSink(transactionId, columnIds, columnTypes, bucketNumber); } private ShardRewriter createShardRewriter(long transactionId, OptionalInt bucketNumber, UUID shardUuid) @@ -594,7 +594,7 @@ private static Map columnIdIndex(List columns) return uniqueIndex(columns, column -> Long.valueOf(column.getColumnName())); } - private class OrcStoragePageSink + private class RaptorStoragePageSink implements StoragePageSink { private final long transactionId; @@ -610,7 +610,7 @@ private class OrcStoragePageSink private OrcFileWriter writer; private UUID shardUuid; - public OrcStoragePageSink(long transactionId, List columnIds, List columnTypes, OptionalInt bucketNumber) + public RaptorStoragePageSink(long transactionId, List columnIds, List columnTypes, OptionalInt bucketNumber) { this.transactionId = transactionId; this.columnIds = ImmutableList.copyOf(requireNonNull(columnIds, "columnIds is null")); diff --git a/plugin/trino-raptor-legacy/src/main/java/io/trino/plugin/raptor/legacy/storage/ShardRecoveryManager.java b/plugin/trino-raptor-legacy/src/main/java/io/trino/plugin/raptor/legacy/storage/ShardRecoveryManager.java index c64a5b644e45..3124ca982c18 100644 --- a/plugin/trino-raptor-legacy/src/main/java/io/trino/plugin/raptor/legacy/storage/ShardRecoveryManager.java +++ b/plugin/trino-raptor-legacy/src/main/java/io/trino/plugin/raptor/legacy/storage/ShardRecoveryManager.java @@ -63,7 +63,7 @@ import static io.trino.plugin.raptor.legacy.RaptorErrorCode.RAPTOR_BACKUP_CORRUPTION; import static io.trino.plugin.raptor.legacy.RaptorErrorCode.RAPTOR_ERROR; import static io.trino.plugin.raptor.legacy.RaptorErrorCode.RAPTOR_RECOVERY_ERROR; -import static io.trino.plugin.raptor.legacy.storage.OrcStorageManager.xxhash64; +import static io.trino.plugin.raptor.legacy.storage.RaptorStorageManager.xxhash64; import static java.nio.file.StandardCopyOption.ATOMIC_MOVE; import static java.util.Objects.requireNonNull; import static java.util.concurrent.Executors.newCachedThreadPool; diff --git a/plugin/trino-raptor-legacy/src/main/java/io/trino/plugin/raptor/legacy/storage/ShardStats.java b/plugin/trino-raptor-legacy/src/main/java/io/trino/plugin/raptor/legacy/storage/ShardStats.java index 1f2bd5403d94..4909d56e2b8c 100644 --- a/plugin/trino-raptor-legacy/src/main/java/io/trino/plugin/raptor/legacy/storage/ShardStats.java +++ b/plugin/trino-raptor-legacy/src/main/java/io/trino/plugin/raptor/legacy/storage/ShardStats.java @@ -39,7 +39,7 @@ import static io.trino.memory.context.AggregatedMemoryContext.newSimpleAggregatedMemoryContext; import static io.trino.orc.OrcReader.INITIAL_BATCH_SIZE; import static io.trino.plugin.raptor.legacy.RaptorErrorCode.RAPTOR_ERROR; -import static io.trino.plugin.raptor.legacy.storage.OrcStorageManager.toOrcFileType; +import static io.trino.plugin.raptor.legacy.storage.RaptorStorageManager.toOrcFileType; import static java.lang.Double.isInfinite; import static java.lang.Double.isNaN; import static org.joda.time.DateTimeZone.UTC; diff --git a/plugin/trino-raptor-legacy/src/main/java/io/trino/plugin/raptor/legacy/storage/StorageModule.java b/plugin/trino-raptor-legacy/src/main/java/io/trino/plugin/raptor/legacy/storage/StorageModule.java index f476ee266403..5c0d0c13ccbd 100644 --- a/plugin/trino-raptor-legacy/src/main/java/io/trino/plugin/raptor/legacy/storage/StorageModule.java +++ b/plugin/trino-raptor-legacy/src/main/java/io/trino/plugin/raptor/legacy/storage/StorageModule.java @@ -49,7 +49,7 @@ public void configure(Binder binder) binder.bind(Ticker.class).toInstance(Ticker.systemTicker()); - binder.bind(StorageManager.class).to(OrcStorageManager.class).in(Scopes.SINGLETON); + binder.bind(StorageManager.class).to(RaptorStorageManager.class).in(Scopes.SINGLETON); binder.bind(StorageService.class).to(FileStorageService.class).in(Scopes.SINGLETON); binder.bind(ShardManager.class).to(DatabaseShardManager.class).in(Scopes.SINGLETON); binder.bind(ShardRecorder.class).to(DatabaseShardRecorder.class).in(Scopes.SINGLETON); @@ -69,7 +69,7 @@ public void configure(Binder binder) newExporter(binder).export(ShardRecoveryManager.class).withGeneratedName(); newExporter(binder).export(BackupManager.class).withGeneratedName(); - newExporter(binder).export(StorageManager.class).as(generator -> generator.generatedNameOf(OrcStorageManager.class)); + newExporter(binder).export(StorageManager.class).as(generator -> generator.generatedNameOf(RaptorStorageManager.class)); newExporter(binder).export(ShardCompactionManager.class).withGeneratedName(); newExporter(binder).export(ShardOrganizer.class).withGeneratedName(); newExporter(binder).export(ShardCompactor.class).withGeneratedName(); diff --git a/plugin/trino-raptor-legacy/src/test/java/io/trino/plugin/raptor/legacy/TestRaptorConnector.java b/plugin/trino-raptor-legacy/src/test/java/io/trino/plugin/raptor/legacy/TestRaptorConnector.java index f54237b68cd6..1108b04261b6 100644 --- a/plugin/trino-raptor-legacy/src/test/java/io/trino/plugin/raptor/legacy/TestRaptorConnector.java +++ b/plugin/trino-raptor-legacy/src/test/java/io/trino/plugin/raptor/legacy/TestRaptorConnector.java @@ -64,7 +64,7 @@ import static io.trino.plugin.raptor.legacy.RaptorTableProperties.TEMPORAL_COLUMN_PROPERTY; import static io.trino.plugin.raptor.legacy.metadata.SchemaDaoUtil.createTablesWithRetry; import static io.trino.plugin.raptor.legacy.metadata.TestDatabaseShardManager.createShardManager; -import static io.trino.plugin.raptor.legacy.storage.TestOrcStorageManager.createOrcStorageManager; +import static io.trino.plugin.raptor.legacy.storage.TestRaptorStorageManager.createRaptorStorageManager; import static io.trino.spi.transaction.IsolationLevel.READ_COMMITTED; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.spi.type.DateType.DATE; @@ -98,7 +98,7 @@ public void setup() NodeManager nodeManager = new TestingNodeManager(); NodeSupplier nodeSupplier = nodeManager::getWorkerNodes; ShardManager shardManager = createShardManager(dbi); - StorageManager storageManager = createOrcStorageManager(dbi, dataDir); + StorageManager storageManager = createRaptorStorageManager(dbi, dataDir); StorageManagerConfig config = new StorageManagerConfig(); connector = new RaptorConnector( new LifeCycleManager(ImmutableList.of(), null), diff --git a/plugin/trino-raptor-legacy/src/test/java/io/trino/plugin/raptor/legacy/storage/TestOrcStorageManager.java b/plugin/trino-raptor-legacy/src/test/java/io/trino/plugin/raptor/legacy/storage/TestRaptorStorageManager.java similarity index 94% rename from plugin/trino-raptor-legacy/src/test/java/io/trino/plugin/raptor/legacy/storage/TestOrcStorageManager.java rename to plugin/trino-raptor-legacy/src/test/java/io/trino/plugin/raptor/legacy/storage/TestRaptorStorageManager.java index 7140bd13592b..22478da37b5d 100644 --- a/plugin/trino-raptor-legacy/src/test/java/io/trino/plugin/raptor/legacy/storage/TestOrcStorageManager.java +++ b/plugin/trino-raptor-legacy/src/test/java/io/trino/plugin/raptor/legacy/storage/TestRaptorStorageManager.java @@ -83,9 +83,9 @@ import static io.trino.metadata.MetadataManager.createTestMetadataManager; import static io.trino.plugin.raptor.legacy.metadata.SchemaDaoUtil.createTablesWithRetry; import static io.trino.plugin.raptor.legacy.metadata.TestDatabaseShardManager.createShardManager; -import static io.trino.plugin.raptor.legacy.storage.OrcStorageManager.xxhash64; import static io.trino.plugin.raptor.legacy.storage.OrcTestingUtil.createReader; import static io.trino.plugin.raptor.legacy.storage.OrcTestingUtil.octets; +import static io.trino.plugin.raptor.legacy.storage.RaptorStorageManager.xxhash64; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.spi.type.BooleanType.BOOLEAN; import static io.trino.spi.type.DateType.DATE; @@ -110,7 +110,7 @@ import static org.testng.FileAssert.assertFile; @Test(singleThreaded = true) -public class TestOrcStorageManager +public class TestRaptorStorageManager { private static final ISOChronology UTC_CHRONOLOGY = ISOChronology.getInstanceUTC(); private static final DateTime EPOCH = new DateTime(0, UTC_CHRONOLOGY); @@ -175,7 +175,7 @@ public void tearDown() public void testWriter() throws Exception { - OrcStorageManager manager = createOrcStorageManager(); + RaptorStorageManager manager = createRaptorStorageManager(); List columnIds = ImmutableList.of(3L, 7L); List columnTypes = ImmutableList.of(BIGINT, createVarcharType(10)); @@ -249,7 +249,7 @@ public void testWriter() public void testReader() throws Exception { - OrcStorageManager manager = createOrcStorageManager(); + RaptorStorageManager manager = createRaptorStorageManager(); List columnIds = ImmutableList.of(2L, 4L, 6L, 7L, 8L, 9L); List columnTypes = ImmutableList.of(BIGINT, createVarcharType(10), VARBINARY, DATE, BOOLEAN, DOUBLE); @@ -323,7 +323,7 @@ public void testReader() public void testRewriter() throws Exception { - OrcStorageManager manager = createOrcStorageManager(); + RaptorStorageManager manager = createRaptorStorageManager(); long transactionId = TRANSACTION_ID; List columnIds = ImmutableList.of(3L, 7L); @@ -373,7 +373,7 @@ public void testWriterRollback() assertEquals(staging.list(), new String[] {}); // create a shard in staging - OrcStorageManager manager = createOrcStorageManager(); + RaptorStorageManager manager = createRaptorStorageManager(); List columnIds = ImmutableList.of(3L, 7L); List columnTypes = ImmutableList.of(BIGINT, createVarcharType(10)); @@ -510,7 +510,7 @@ public void testShardStatsDateTimestamp() @Test public void testMaxShardRows() { - OrcStorageManager manager = createOrcStorageManager(2, DataSize.of(2, MEGABYTE)); + RaptorStorageManager manager = createRaptorStorageManager(2, DataSize.of(2, MEGABYTE)); List columnIds = ImmutableList.of(3L, 7L); List columnTypes = ImmutableList.of(BIGINT, createVarcharType(10)); @@ -536,14 +536,14 @@ public void testMaxFileSize() .build(); // Set maxFileSize to 1 byte, so adding any page makes the StoragePageSink full - OrcStorageManager manager = createOrcStorageManager(20, DataSize.ofBytes(1)); + RaptorStorageManager manager = createRaptorStorageManager(20, DataSize.ofBytes(1)); StoragePageSink sink = createStoragePageSink(manager, columnIds, columnTypes); sink.appendPages(pages); assertTrue(sink.isFull()); } private static ConnectorPageSource getPageSource( - OrcStorageManager manager, + RaptorStorageManager manager, List columnIds, List columnTypes, UUID uuid, @@ -558,22 +558,22 @@ private static StoragePageSink createStoragePageSink(StorageManager manager, Lis return manager.createStoragePageSink(transactionId, OptionalInt.empty(), columnIds, columnTypes, false); } - private OrcStorageManager createOrcStorageManager() + private RaptorStorageManager createRaptorStorageManager() { - return createOrcStorageManager(MAX_SHARD_ROWS, MAX_FILE_SIZE); + return createRaptorStorageManager(MAX_SHARD_ROWS, MAX_FILE_SIZE); } - private OrcStorageManager createOrcStorageManager(int maxShardRows, DataSize maxFileSize) + private RaptorStorageManager createRaptorStorageManager(int maxShardRows, DataSize maxFileSize) { - return createOrcStorageManager(storageService, backupStore, recoveryManager, shardRecorder, maxShardRows, maxFileSize); + return createRaptorStorageManager(storageService, backupStore, recoveryManager, shardRecorder, maxShardRows, maxFileSize); } - public static OrcStorageManager createOrcStorageManager(IDBI dbi, File temporary) + public static RaptorStorageManager createRaptorStorageManager(IDBI dbi, File temporary) { - return createOrcStorageManager(dbi, temporary, MAX_SHARD_ROWS); + return createRaptorStorageManager(dbi, temporary, MAX_SHARD_ROWS); } - public static OrcStorageManager createOrcStorageManager(IDBI dbi, File temporary, int maxShardRows) + public static RaptorStorageManager createRaptorStorageManager(IDBI dbi, File temporary, int maxShardRows) { File directory = new File(temporary, "data"); StorageService storageService = new FileStorageService(directory); @@ -592,7 +592,7 @@ public static OrcStorageManager createOrcStorageManager(IDBI dbi, File temporary shardManager, MISSING_SHARD_DISCOVERY, 10); - return createOrcStorageManager( + return createRaptorStorageManager( storageService, backupStore, recoveryManager, @@ -601,7 +601,7 @@ public static OrcStorageManager createOrcStorageManager(IDBI dbi, File temporary MAX_FILE_SIZE); } - public static OrcStorageManager createOrcStorageManager( + public static RaptorStorageManager createRaptorStorageManager( StorageService storageService, Optional backupStore, ShardRecoveryManager recoveryManager, @@ -609,7 +609,7 @@ public static OrcStorageManager createOrcStorageManager( int maxShardRows, DataSize maxFileSize) { - return new OrcStorageManager( + return new RaptorStorageManager( CURRENT_NODE, storageService, backupStore, @@ -669,7 +669,7 @@ private List columnStats(List columnTypes, Object[]... rows) } List columnIds = list.build(); - OrcStorageManager manager = createOrcStorageManager(); + RaptorStorageManager manager = createRaptorStorageManager(); StoragePageSink sink = createStoragePageSink(manager, columnIds, columnTypes); sink.appendPages(rowPagesBuilder(columnTypes).rows(rows).build()); List shards = getFutureValue(sink.commit()); diff --git a/plugin/trino-raptor-legacy/src/test/java/io/trino/plugin/raptor/legacy/storage/TestShardRecovery.java b/plugin/trino-raptor-legacy/src/test/java/io/trino/plugin/raptor/legacy/storage/TestShardRecovery.java index 45da969bfd7f..f7ee66281451 100644 --- a/plugin/trino-raptor-legacy/src/test/java/io/trino/plugin/raptor/legacy/storage/TestShardRecovery.java +++ b/plugin/trino-raptor-legacy/src/test/java/io/trino/plugin/raptor/legacy/storage/TestShardRecovery.java @@ -42,7 +42,7 @@ import static io.trino.plugin.raptor.legacy.RaptorErrorCode.RAPTOR_BACKUP_CORRUPTION; import static io.trino.plugin.raptor.legacy.metadata.SchemaDaoUtil.createTablesWithRetry; import static io.trino.plugin.raptor.legacy.metadata.TestDatabaseShardManager.createShardManager; -import static io.trino.plugin.raptor.legacy.storage.OrcStorageManager.xxhash64; +import static io.trino.plugin.raptor.legacy.storage.RaptorStorageManager.xxhash64; import static io.trino.testing.assertions.TrinoExceptionAssert.assertTrinoExceptionThrownBy; import static java.io.File.createTempFile; import static java.lang.String.format; diff --git a/plugin/trino-raptor-legacy/src/test/java/io/trino/plugin/raptor/legacy/storage/organization/TestShardCompactor.java b/plugin/trino-raptor-legacy/src/test/java/io/trino/plugin/raptor/legacy/storage/organization/TestShardCompactor.java index 2b505c0cb96c..a98f5f1ac430 100644 --- a/plugin/trino-raptor-legacy/src/test/java/io/trino/plugin/raptor/legacy/storage/organization/TestShardCompactor.java +++ b/plugin/trino-raptor-legacy/src/test/java/io/trino/plugin/raptor/legacy/storage/organization/TestShardCompactor.java @@ -21,7 +21,7 @@ import io.trino.orc.OrcReaderOptions; import io.trino.plugin.raptor.legacy.metadata.ColumnInfo; import io.trino.plugin.raptor.legacy.metadata.ShardInfo; -import io.trino.plugin.raptor.legacy.storage.OrcStorageManager; +import io.trino.plugin.raptor.legacy.storage.RaptorStorageManager; import io.trino.plugin.raptor.legacy.storage.StorageManager; import io.trino.plugin.raptor.legacy.storage.StoragePageSink; import io.trino.spi.Page; @@ -54,7 +54,7 @@ import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; import static io.airlift.concurrent.MoreFutures.getFutureValue; import static io.airlift.units.DataSize.Unit.MEGABYTE; -import static io.trino.plugin.raptor.legacy.storage.TestOrcStorageManager.createOrcStorageManager; +import static io.trino.plugin.raptor.legacy.storage.TestRaptorStorageManager.createRaptorStorageManager; import static io.trino.spi.connector.SortOrder.ASC_NULLS_FIRST; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.spi.type.DateType.DATE; @@ -80,7 +80,7 @@ public class TestShardCompactor .withStreamBufferSize(DataSize.of(1, MEGABYTE)) .withTinyStripeThreshold(DataSize.of(1, MEGABYTE)); - private OrcStorageManager storageManager; + private RaptorStorageManager storageManager; private ShardCompactor compactor; private File temporary; private Handle dummyHandle; @@ -91,7 +91,7 @@ public void setup() temporary = createTempDir(); IDBI dbi = new DBI("jdbc:h2:mem:test" + System.nanoTime() + ThreadLocalRandom.current().nextLong()); dummyHandle = dbi.open(); - storageManager = createOrcStorageManager(dbi, temporary, MAX_SHARD_ROWS); + storageManager = createRaptorStorageManager(dbi, temporary, MAX_SHARD_ROWS); compactor = new ShardCompactor(storageManager, READER_OPTIONS, new TypeOperators()); } diff --git a/plugin/trino-redis/pom.xml b/plugin/trino-redis/pom.xml index e673f54580ef..82fddf14d896 100644 --- a/plugin/trino-redis/pom.xml +++ b/plugin/trino-redis/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-redis/src/main/java/io/trino/plugin/redis/RedisConnectorConfig.java b/plugin/trino-redis/src/main/java/io/trino/plugin/redis/RedisConnectorConfig.java index 01a81cecd6d0..3fe0a0e3744f 100644 --- a/plugin/trino-redis/src/main/java/io/trino/plugin/redis/RedisConnectorConfig.java +++ b/plugin/trino-redis/src/main/java/io/trino/plugin/redis/RedisConnectorConfig.java @@ -29,6 +29,7 @@ import java.util.stream.StreamSupport; import static com.google.common.collect.ImmutableSet.toImmutableSet; +import static java.util.concurrent.TimeUnit.MILLISECONDS; public class RedisConnectorConfig { @@ -62,7 +63,7 @@ public class RedisConnectorConfig /** * Timeout to connect to Redis. */ - private Duration redisConnectTimeout = Duration.valueOf("2000ms"); + private Duration redisConnectTimeout = new Duration(2000, MILLISECONDS); /** * The schema name to use in the connector. diff --git a/plugin/trino-redis/src/test/java/io/trino/plugin/redis/TestRedisConnectorTest.java b/plugin/trino-redis/src/test/java/io/trino/plugin/redis/TestRedisConnectorTest.java index 31c2de7315fe..8fd8cdde4b18 100644 --- a/plugin/trino-redis/src/test/java/io/trino/plugin/redis/TestRedisConnectorTest.java +++ b/plugin/trino-redis/src/test/java/io/trino/plugin/redis/TestRedisConnectorTest.java @@ -45,6 +45,9 @@ public void destroy() protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior) { switch (connectorBehavior) { + case SUPPORTS_TOPN_PUSHDOWN: + return false; + case SUPPORTS_CREATE_SCHEMA: return false; diff --git a/plugin/trino-redshift/pom.xml b/plugin/trino-redshift/pom.xml index fbaa1b1bfa9e..c4abf93fd9fa 100644 --- a/plugin/trino-redshift/pom.xml +++ b/plugin/trino-redshift/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-resource-group-managers/pom.xml b/plugin/trino-resource-group-managers/pom.xml index 58cf5b665149..7ca367dd671e 100644 --- a/plugin/trino-resource-group-managers/pom.xml +++ b/plugin/trino-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-resource-group-managers/src/test/java/io/trino/plugin/resourcegroups/TestStaticSelector.java b/plugin/trino-resource-group-managers/src/test/java/io/trino/plugin/resourcegroups/TestStaticSelector.java index 0076c6d2c8f7..0142361b7952 100644 --- a/plugin/trino-resource-group-managers/src/test/java/io/trino/plugin/resourcegroups/TestStaticSelector.java +++ b/plugin/trino-resource-group-managers/src/test/java/io/trino/plugin/resourcegroups/TestStaticSelector.java @@ -30,6 +30,7 @@ import static io.airlift.units.DataSize.Unit.MEGABYTE; import static io.airlift.units.DataSize.Unit.TERABYTE; +import static java.util.concurrent.TimeUnit.MINUTES; import static org.testng.Assert.assertEquals; public class TestStaticSelector @@ -101,7 +102,7 @@ public void testSelectorResourceEstimate() Optional.of(new SelectorResourceEstimate( Optional.of(new Range<>( Optional.empty(), - Optional.of(Duration.valueOf("5m")))), + Optional.of(new Duration(5, MINUTES)))), Optional.empty(), Optional.of(new Range<>( Optional.empty(), diff --git a/plugin/trino-resource-group-managers/src/test/java/io/trino/plugin/resourcegroups/db/TestDbResourceGroupConfigurationManager.java b/plugin/trino-resource-group-managers/src/test/java/io/trino/plugin/resourcegroups/db/TestDbResourceGroupConfigurationManager.java index bb1b84597b44..0018c89337c3 100644 --- a/plugin/trino-resource-group-managers/src/test/java/io/trino/plugin/resourcegroups/db/TestDbResourceGroupConfigurationManager.java +++ b/plugin/trino-resource-group-managers/src/test/java/io/trino/plugin/resourcegroups/db/TestDbResourceGroupConfigurationManager.java @@ -297,7 +297,7 @@ public void testInvalidConfiguration() DbResourceGroupConfigurationManager manager = new DbResourceGroupConfigurationManager( (poolId, listener) -> {}, - new DbResourceGroupConfig().setMaxRefreshInterval(io.airlift.units.Duration.valueOf("1ms")), + new DbResourceGroupConfig().setMaxRefreshInterval(new io.airlift.units.Duration(1, MILLISECONDS)), daoProvider.get(), ENVIRONMENT); @@ -317,7 +317,7 @@ public void testRefreshInterval() DbResourceGroupConfigurationManager manager = new DbResourceGroupConfigurationManager( (poolId, listener) -> {}, - new DbResourceGroupConfig().setMaxRefreshInterval(io.airlift.units.Duration.valueOf("1ms")), + new DbResourceGroupConfig().setMaxRefreshInterval(new io.airlift.units.Duration(1, MILLISECONDS)), daoProvider.get(), ENVIRONMENT); diff --git a/plugin/trino-resource-group-managers/src/test/java/io/trino/plugin/resourcegroups/db/TestResourceGroupsDao.java b/plugin/trino-resource-group-managers/src/test/java/io/trino/plugin/resourcegroups/db/TestResourceGroupsDao.java index 089f4d479880..dd3a7d257771 100644 --- a/plugin/trino-resource-group-managers/src/test/java/io/trino/plugin/resourcegroups/db/TestResourceGroupsDao.java +++ b/plugin/trino-resource-group-managers/src/test/java/io/trino/plugin/resourcegroups/db/TestResourceGroupsDao.java @@ -37,6 +37,9 @@ import static io.trino.spi.resourcegroups.QueryType.EXPLAIN; import static io.trino.spi.resourcegroups.QueryType.INSERT; import static io.trino.spi.resourcegroups.QueryType.SELECT; +import static java.util.concurrent.TimeUnit.HOURS; +import static java.util.concurrent.TimeUnit.MINUTES; +import static java.util.concurrent.TimeUnit.SECONDS; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; @@ -47,9 +50,9 @@ public class TestResourceGroupsDao private static final SelectorResourceEstimate SELECTOR_RESOURCE_ESTIMATE = new SelectorResourceEstimate( Optional.of(new Range<>( Optional.empty(), - Optional.of(Duration.valueOf("5m")))), + Optional.of(new Duration(5, MINUTES)))), Optional.of(new SelectorResourceEstimate.Range<>( - Optional.of(Duration.valueOf("10s")), + Optional.of(new Duration(10, SECONDS)), Optional.empty())), Optional.of(new Range<>( Optional.empty(), @@ -237,7 +240,7 @@ public void testGlobalResourceGroupProperties() H2ResourceGroupsDao dao = setup("global_properties"); dao.createResourceGroupsGlobalPropertiesTable(); dao.insertResourceGroupsGlobalProperties("cpu_quota_period", "1h"); - ResourceGroupGlobalProperties globalProperties = new ResourceGroupGlobalProperties(Optional.of(Duration.valueOf("1h"))); + ResourceGroupGlobalProperties globalProperties = new ResourceGroupGlobalProperties(Optional.of(new Duration(1, HOURS))); ResourceGroupGlobalProperties records = dao.getResourceGroupGlobalProperties().get(0); assertEquals(globalProperties, records); try { diff --git a/plugin/trino-session-property-managers/pom.xml b/plugin/trino-session-property-managers/pom.xml index 01672ac36e2a..459b2d5e27ca 100644 --- a/plugin/trino-session-property-managers/pom.xml +++ b/plugin/trino-session-property-managers/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-sqlserver/pom.xml b/plugin/trino-sqlserver/pom.xml index 71d3248068be..11aafa8d30cc 100644 --- a/plugin/trino-sqlserver/pom.xml +++ b/plugin/trino-sqlserver/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml @@ -28,6 +28,11 @@ trino-matching + + io.airlift + configuration + + io.airlift log diff --git a/plugin/trino-sqlserver/src/main/java/io/trino/plugin/sqlserver/SqlServerClient.java b/plugin/trino-sqlserver/src/main/java/io/trino/plugin/sqlserver/SqlServerClient.java index 08843bf79fa2..d1a3fa57c9db 100644 --- a/plugin/trino-sqlserver/src/main/java/io/trino/plugin/sqlserver/SqlServerClient.java +++ b/plugin/trino-sqlserver/src/main/java/io/trino/plugin/sqlserver/SqlServerClient.java @@ -130,6 +130,7 @@ import static java.lang.String.join; import static java.math.RoundingMode.UNNECESSARY; import static java.time.Duration.ofMinutes; +import static java.util.Objects.requireNonNull; import static java.util.stream.Collectors.joining; public class SqlServerClient @@ -140,6 +141,7 @@ public class SqlServerClient private static final Joiner DOT_JOINER = Joiner.on("."); + private final boolean snapshotIsolationDisabled; private final Cache snapshotIsolationEnabled = CacheBuilder.newBuilder() .maximumSize(1) .expireAfterWrite(ofMinutes(5)) @@ -150,10 +152,13 @@ public class SqlServerClient private static final int MAX_SUPPORTED_TEMPORAL_PRECISION = 7; @Inject - public SqlServerClient(BaseJdbcConfig config, ConnectionFactory connectionFactory) + public SqlServerClient(BaseJdbcConfig config, SqlServerConfig sqlServerConfig, ConnectionFactory connectionFactory) { super(config, "\"", connectionFactory); + requireNonNull(sqlServerConfig, "sqlServerConfig is null"); + snapshotIsolationDisabled = sqlServerConfig.isSnapshotIsolationDisabled(); + JdbcTypeHandle bigintTypeHandle = new JdbcTypeHandle(Types.BIGINT, Optional.of("bigint"), Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty()); this.aggregateFunctionRewriter = new AggregateFunctionRewriter( this::quoted, @@ -538,6 +543,9 @@ public Connection getConnection(ConnectorSession session, JdbcSplit split) private Connection configureConnectionTransactionIsolation(Connection connection) throws SQLException { + if (snapshotIsolationDisabled) { + return connection; + } try { if (hasSnapshotIsolationEnabled(connection)) { // SQL Server's READ COMMITTED + SNAPSHOT ISOLATION is equivalent to ordinary READ COMMITTED in e.g. Oracle, PostgreSQL. @@ -558,7 +566,7 @@ private boolean hasSnapshotIsolationEnabled(Connection connection) try { return snapshotIsolationEnabled.get(SnapshotIsolationEnabledCacheKey.INSTANCE, () -> { Handle handle = Jdbi.open(connection); - return handle.createQuery("SELECT is_read_committed_snapshot_on FROM sys.databases WHERE name = :name") + return handle.createQuery("SELECT snapshot_isolation_state FROM sys.databases WHERE name = :name") .bind("name", connection.getCatalog()) .mapTo(Boolean.class) .findOne() @@ -589,7 +597,7 @@ public static ColumnMapping varbinaryColumnMapping() DISABLE_PUSHDOWN); } - private static SliceWriteFunction varbinaryWriteFunction() + public static SliceWriteFunction varbinaryWriteFunction() { return new SliceWriteFunction() { diff --git a/plugin/trino-sqlserver/src/main/java/io/trino/plugin/sqlserver/SqlServerClientModule.java b/plugin/trino-sqlserver/src/main/java/io/trino/plugin/sqlserver/SqlServerClientModule.java index 6e6e39231fe0..7e2791ff77cd 100644 --- a/plugin/trino-sqlserver/src/main/java/io/trino/plugin/sqlserver/SqlServerClientModule.java +++ b/plugin/trino-sqlserver/src/main/java/io/trino/plugin/sqlserver/SqlServerClientModule.java @@ -29,6 +29,7 @@ import io.trino.plugin.jdbc.credential.CredentialProvider; import static com.google.inject.multibindings.OptionalBinder.newOptionalBinder; +import static io.airlift.configuration.ConfigBinder.configBinder; import static io.trino.plugin.jdbc.JdbcModule.bindTablePropertiesProvider; import static io.trino.plugin.sqlserver.SqlServerClient.SQL_SERVER_MAX_LIST_EXPRESSIONS; @@ -38,6 +39,7 @@ public class SqlServerClientModule @Override public void configure(Binder binder) { + configBinder(binder).bindConfig(SqlServerConfig.class); binder.bind(JdbcClient.class).annotatedWith(ForBaseJdbc.class).to(SqlServerClient.class).in(Scopes.SINGLETON); bindTablePropertiesProvider(binder, SqlServerTableProperties.class); newOptionalBinder(binder, Key.get(int.class, MaxDomainCompactionThreshold.class)).setBinding().toInstance(SQL_SERVER_MAX_LIST_EXPRESSIONS); diff --git a/plugin/trino-sqlserver/src/main/java/io/trino/plugin/sqlserver/SqlServerConfig.java b/plugin/trino-sqlserver/src/main/java/io/trino/plugin/sqlserver/SqlServerConfig.java new file mode 100644 index 000000000000..340de7e01b78 --- /dev/null +++ b/plugin/trino-sqlserver/src/main/java/io/trino/plugin/sqlserver/SqlServerConfig.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 io.trino.plugin.sqlserver; + +import io.airlift.configuration.Config; +import io.airlift.configuration.ConfigDescription; + +public class SqlServerConfig +{ + private boolean snapshotIsolationDisabled; + + public boolean isSnapshotIsolationDisabled() + { + return snapshotIsolationDisabled; + } + + @Config("sqlserver.snapshot-isolation.disabled") + @ConfigDescription("Disables automatic use of snapshot isolation for transactions issued by Trino in SQL Server") + public SqlServerConfig setSnapshotIsolationDisabled(boolean snapshotIsolationDisabled) + { + this.snapshotIsolationDisabled = snapshotIsolationDisabled; + return this; + } +} diff --git a/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/BaseSqlServerTransactionIsolationTest.java b/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/BaseSqlServerTransactionIsolationTest.java new file mode 100644 index 000000000000..09101c426617 --- /dev/null +++ b/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/BaseSqlServerTransactionIsolationTest.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 io.trino.plugin.sqlserver; + +import io.trino.testing.AbstractTestQueryFramework; +import io.trino.testing.MaterializedResult; +import io.trino.testing.QueryRunner; +import org.testng.annotations.Test; + +import java.util.List; +import java.util.Map; + +import static io.trino.plugin.sqlserver.SqlServerQueryRunner.createSqlServerQueryRunner; +import static io.trino.spi.type.VarcharType.VARCHAR; +import static io.trino.tpch.TpchTable.NATION; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +public abstract class BaseSqlServerTransactionIsolationTest + extends AbstractTestQueryFramework +{ + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + TestingSqlServer sqlServer = closeAfterClass(new TestingSqlServer()); + sqlServer.start(); + configureDatabase(sqlServer); + return createSqlServerQueryRunner( + sqlServer, + Map.of(), + Map.of(), + List.of(NATION)); + } + + protected abstract void configureDatabase(TestingSqlServer sqlServer); + + @Test + public void testCreateReadTable() + { + assertUpdate("CREATE TABLE ctas_read AS SELECT * FROM tpch.tiny.nation", "SELECT count(*) FROM nation"); + assertQuery("SELECT AVG(LENGTH(name)) FROM ctas_read", "SELECT 7.08"); + assertQuery("SELECT SUM(LENGTH(name)) FROM ctas_read WHERE regionkey = 1", "SELECT 38"); + assertUpdate("DROP TABLE ctas_read"); + } + + @Test + public void testDescribeShowTable() + { + assertUpdate("CREATE TABLE ctas_describe AS SELECT regionkey, nationkey, comment FROM tpch.tiny.nation", "SELECT count(*) FROM nation"); + + MaterializedResult expectedColumns = MaterializedResult.resultBuilder(getQueryRunner().getDefaultSession(), VARCHAR, VARCHAR, VARCHAR, VARCHAR) + .row("regionkey", "bigint", "", "") + .row("nationkey", "bigint", "", "") + .row("comment", "varchar(152)", "", "") + .build(); + + MaterializedResult actualColumns = computeActual("DESCRIBE ctas_describe"); + assertThat(actualColumns).isEqualTo(expectedColumns); + + MaterializedResult expectedTables = MaterializedResult.resultBuilder(getQueryRunner().getDefaultSession(), VARCHAR) + .row("ctas_describe") + .build(); + + MaterializedResult actualTables = computeActual("SHOW TABLES LIKE 'ctas_describe'"); + assertThat(actualTables).isEqualTo(expectedTables); + + assertUpdate("DROP TABLE ctas_describe"); + } + + @Test + public void testCreateInsertReadTable() + { + assertUpdate("CREATE TABLE insert_table (col INTEGER)"); + assertUpdate("INSERT INTO insert_table (col) VALUES (1), (2), (3), (4)", 4); + assertQuery("SELECT AVG(col) FROM insert_table", "SELECT 2.5"); + assertUpdate("DROP TABLE insert_table"); + } +} diff --git a/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/BaseSqlServerTypeMapping.java b/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/BaseSqlServerTypeMapping.java new file mode 100644 index 000000000000..b739d47cae36 --- /dev/null +++ b/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/BaseSqlServerTypeMapping.java @@ -0,0 +1,607 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.trino.plugin.sqlserver; + +import com.google.common.collect.ImmutableList; +import io.trino.Session; +import io.trino.spi.type.TimeZoneKey; +import io.trino.testing.AbstractTestQueryFramework; +import io.trino.testing.TestingSession; +import io.trino.testing.datatype.CreateAndInsertDataSetup; +import io.trino.testing.datatype.CreateAsSelectDataSetup; +import io.trino.testing.datatype.DataSetup; +import io.trino.testing.datatype.SqlDataTypeTest; +import io.trino.testing.sql.SqlExecutor; +import io.trino.testing.sql.TrinoSqlExecutor; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.ZoneId; + +import static com.google.common.base.Preconditions.checkState; +import static com.google.common.base.Verify.verify; +import static io.trino.spi.type.BigintType.BIGINT; +import static io.trino.spi.type.BooleanType.BOOLEAN; +import static io.trino.spi.type.CharType.createCharType; +import static io.trino.spi.type.DateType.DATE; +import static io.trino.spi.type.DecimalType.createDecimalType; +import static io.trino.spi.type.DoubleType.DOUBLE; +import static io.trino.spi.type.IntegerType.INTEGER; +import static io.trino.spi.type.RealType.REAL; +import static io.trino.spi.type.SmallintType.SMALLINT; +import static io.trino.spi.type.TimeType.createTimeType; +import static io.trino.spi.type.TimeZoneKey.UTC_KEY; +import static io.trino.spi.type.TimestampType.createTimestampType; +import static io.trino.spi.type.TinyintType.TINYINT; +import static io.trino.spi.type.VarbinaryType.VARBINARY; +import static io.trino.spi.type.VarcharType.createUnboundedVarcharType; +import static io.trino.spi.type.VarcharType.createVarcharType; +import static java.time.ZoneOffset.UTC; + +public abstract class BaseSqlServerTypeMapping + extends AbstractTestQueryFramework +{ + private final ZoneId jvmZone = ZoneId.systemDefault(); + private final LocalDateTime timeGapInJvmZone1 = LocalDateTime.of(1970, 1, 1, 0, 13, 42); + private final LocalDateTime timeGapInJvmZone2 = LocalDateTime.of(2018, 4, 1, 2, 13, 55, 123_000_000); + private final LocalDateTime timeDoubledInJvmZone = LocalDateTime.of(2018, 10, 28, 1, 33, 17, 456_000_000); + + // no DST in 1970, but has DST in later years (e.g. 2018) + private final ZoneId vilnius = ZoneId.of("Europe/Vilnius"); + private final LocalDateTime timeGapInVilnius = LocalDateTime.of(2018, 3, 25, 3, 17, 17); + private final LocalDateTime timeDoubledInVilnius = LocalDateTime.of(2018, 10, 28, 3, 33, 33, 333_000_000); + + // minutes offset change since 1970-01-01, no DST + private final ZoneId kathmandu = ZoneId.of("Asia/Kathmandu"); + private final LocalDateTime timeGapInKathmandu = LocalDateTime.of(1986, 1, 1, 0, 13, 7); + + @BeforeClass + public void setUp() + { + checkIsGap(jvmZone, timeGapInJvmZone1); + checkIsGap(jvmZone, timeGapInJvmZone2); + checkIsDoubled(jvmZone, timeDoubledInJvmZone); + + checkIsGap(vilnius, timeGapInVilnius); + checkIsDoubled(vilnius, timeDoubledInVilnius); + + checkIsGap(kathmandu, timeGapInKathmandu); + } + + @Test + public void testBasicTypes() + { + SqlDataTypeTest.create() + .addRoundTrip("boolean", "null", BOOLEAN, "CAST(NULL AS BOOLEAN)") + .addRoundTrip("boolean", "true", BOOLEAN) + .addRoundTrip("boolean", "false", BOOLEAN) + .addRoundTrip("bigint", "null", BIGINT, "CAST(NULL AS BIGINT)") + .addRoundTrip("bigint", "123456789012", BIGINT) + .addRoundTrip("integer", "null", INTEGER, "CAST(NULL AS INTEGER)") + .addRoundTrip("integer", "123456789", INTEGER) + .addRoundTrip("smallint", "null", SMALLINT, "CAST(NULL AS SMALLINT)") + .addRoundTrip("smallint", "32456", SMALLINT, "SMALLINT '32456'") + .addRoundTrip("tinyint", "null", TINYINT, "CAST(NULL AS TINYINT)") + .addRoundTrip("tinyint", "5", TINYINT, "TINYINT '5'") + .execute(getQueryRunner(), trinoCreateAsSelect("test_basic_types")); + } + + @Test + public void testReal() + { + // we are not testing Nan/-Infinity/+Infinity as those are not supported by SQL Server + SqlDataTypeTest.create() + .addRoundTrip("real", "NULL", REAL, "CAST(NULL AS real)") + .addRoundTrip("real", "3.14", REAL, "REAL '3.14'") + .addRoundTrip("real", "3.1415927", REAL, "REAL '3.1415927'") + .execute(getQueryRunner(), sqlServerCreateAndInsert("test_real")); + + SqlDataTypeTest.create() + .addRoundTrip("real", "NULL", REAL, "CAST(NULL AS real)") + .addRoundTrip("real", "3.14", REAL, "REAL '3.14'") + .addRoundTrip("real", "3.1415927", REAL, "REAL '3.1415927'") + .execute(getQueryRunner(), trinoCreateAsSelect("test_real")); + } + + @Test + public void testDouble() + { + // we are not testing Nan/-Infinity/+Infinity as those are not supported by SQL Server + SqlDataTypeTest.create() + .addRoundTrip("double precision", "NULL", DOUBLE, "CAST(NULL AS double)") + .addRoundTrip("double precision", "1.0E100", DOUBLE, "1.0E100") + .addRoundTrip("double precision", "123.456E10", DOUBLE, "123.456E10") + .execute(getQueryRunner(), sqlServerCreateAndInsert("test_double")); + + SqlDataTypeTest.create() + .addRoundTrip("double", "NULL", DOUBLE, "CAST(NULL AS double)") + .addRoundTrip("double", "1.0E100", DOUBLE, "1.0E100") + .addRoundTrip("double", "123.456E10", DOUBLE, "123.456E10") + .execute(getQueryRunner(), trinoCreateAsSelect("test_double")); + } + + @Test + public void testDecimal() + { + SqlDataTypeTest.create() + .addRoundTrip("decimal(3, 0)", "193", createDecimalType(3, 0), "CAST('193' AS decimal(3, 0))") + .addRoundTrip("decimal(3, 0)", "19", createDecimalType(3, 0), "CAST('19' AS decimal(3, 0))") + .addRoundTrip("decimal(3, 0)", "-193", createDecimalType(3, 0), "CAST('-193' AS decimal(3, 0))") + .addRoundTrip("decimal(3, 1)", "10.0", createDecimalType(3, 1), "CAST('10.0' AS decimal(3, 1))") + .addRoundTrip("decimal(3, 1)", "10.1", createDecimalType(3, 1), "CAST('10.1' AS decimal(3, 1))") + .addRoundTrip("decimal(3, 1)", "-10.1", createDecimalType(3, 1), "CAST('-10.1' AS decimal(3, 1))") + .addRoundTrip("decimal(4, 2)", "2", createDecimalType(4, 2), "CAST('2' AS decimal(4, 2))") + .addRoundTrip("decimal(4, 2)", "2.3", createDecimalType(4, 2), "CAST('2.3' AS decimal(4, 2))") + .addRoundTrip("decimal(24, 2)", "2", createDecimalType(24, 2), "CAST('2' AS decimal(24, 2))") + .addRoundTrip("decimal(24, 2)", "2.3", createDecimalType(24, 2), "CAST('2.3' AS decimal(24, 2))") + .addRoundTrip("decimal(24, 2)", "123456789.3", createDecimalType(24, 2), "CAST('123456789.3' AS decimal(24, 2))") + .addRoundTrip("decimal(24, 4)", "12345678901234567890.31", createDecimalType(24, 4), "CAST('12345678901234567890.31' AS decimal(24, 4))") + .addRoundTrip("decimal(30, 5)", "3141592653589793238462643.38327", createDecimalType(30, 5), "CAST('3141592653589793238462643.38327' AS decimal(30, 5))") + .addRoundTrip("decimal(30, 5)", "-3141592653589793238462643.38327", createDecimalType(30, 5), "CAST('-3141592653589793238462643.38327' AS decimal(30, 5))") + .execute(getQueryRunner(), sqlServerCreateAndInsert("test_decimal")) + .execute(getQueryRunner(), trinoCreateAsSelect("test_decimal")); + } + + @Test + public void testChar() + { + SqlDataTypeTest.create() + .addRoundTrip("char(1)", "NULL", createCharType(1), "CAST(NULL AS char(1))") + .addRoundTrip("char(10)", "'text_a'", createCharType(10), "CAST('text_a' AS char(10))") + .addRoundTrip("char(255)", "'text_b'", createCharType(255), "CAST('text_b' AS char(255))") + .addRoundTrip("char(4001)", "'text_c'", createCharType(4001), "CAST('text_c' AS char(4001))") + .execute(getQueryRunner(), sqlServerCreateAndInsert("test_char")); + + SqlDataTypeTest.create() + .addRoundTrip("char(1)", "NULL", createCharType(1), "CAST(NULL AS char(1))") + .addRoundTrip("char(10)", "'text_a'", createCharType(10), "CAST('text_a' AS char(10))") + .addRoundTrip("char(255)", "'text_b'", createCharType(255), "CAST('text_b' AS char(255))") + .addRoundTrip("char(5)", "CAST('攻殻機動隊' AS char(5))", createCharType(5), "CAST('攻殻機動隊' AS char(5))") + .addRoundTrip("char(32)", "CAST('攻殻機動隊' AS char(32))", createCharType(32), "CAST('攻殻機動隊' AS char(32))") + .addRoundTrip("char(20)", "CAST('😂' AS char(20))", createCharType(20), "CAST('😂' AS char(20))") + .addRoundTrip("char(77)", "CAST('Ну, погоди!' AS char(77))", createCharType(77), "CAST('Ну, погоди!' AS char(77))") + // testing mapping char > 4000 -> varchar(max) + .addRoundTrip("char(4001)", "'text_c'", createUnboundedVarcharType(), "CAST('text_c' AS varchar)") + .execute(getQueryRunner(), trinoCreateAndInsert(getSession(), "test_char")) + .execute(getQueryRunner(), trinoCreateAsSelect("test_char")); + } + + @Test + public void testSqlServerNchar() + { + SqlDataTypeTest.create() + .addRoundTrip("nchar(1)", "NULL", createCharType(1), "CAST(NULL AS char(1))") + .addRoundTrip("nchar(10)", "'text_a'", createCharType(10), "CAST('text_a' AS char(10))") + .addRoundTrip("nchar(255)", "'text_b'", createCharType(255), "CAST('text_b' AS char(255))") + .execute(getQueryRunner(), sqlServerCreateAndInsert("test_sqlserver_char")); + } + + @Test + public void testTrinoLongChar() + { + // testing mapping char > 4000 -> varchar(max) + SqlDataTypeTest.create() + .addRoundTrip("char(4001)", "'text_c'", createUnboundedVarcharType(), "CAST('text_c' AS varchar)") + .execute(getQueryRunner(), trinoCreateAndInsert(getSession(), "test_long_char")) + .execute(getQueryRunner(), trinoCreateAsSelect("test_long_char")); + } + + @Test + public void testVarchar() + { + SqlDataTypeTest.create() + .addRoundTrip("varchar(10)", "'text_a'", createVarcharType(10), "CAST('text_a' AS varchar(10))") + .addRoundTrip("varchar(255)", "'text_b'", createVarcharType(255), "CAST('text_b' AS varchar(255))") + .addRoundTrip("varchar(4001)", "'text_c'", createVarcharType(4001), "CAST('text_c' AS varchar(4001))") + .execute(getQueryRunner(), sqlServerCreateAndInsert("test_varchar")); + + SqlDataTypeTest.create() + .addRoundTrip("varchar(10)", "'text_a'", createVarcharType(10), "CAST('text_a' AS varchar(10))") + .addRoundTrip("varchar(255)", "'text_b'", createVarcharType(255), "CAST('text_b' AS varchar(255))") + .addRoundTrip("varchar(5)", "CAST('攻殻機動隊' AS varchar(5))", createVarcharType(5), "CAST('攻殻機動隊' AS varchar(5))") + .addRoundTrip("varchar(32)", "CAST('攻殻機動隊' AS varchar(32))", createVarcharType(32), "CAST('攻殻機動隊' AS varchar(32))") + .addRoundTrip("varchar(20)", "CAST('😂' AS varchar(20))", createVarcharType(20), "CAST('😂' AS varchar(20))") + .addRoundTrip("varchar(77)", "CAST('Ну, погоди!' AS varchar(77))", createVarcharType(77), "CAST('Ну, погоди!' AS varchar(77))") + .execute(getQueryRunner(), trinoCreateAndInsert(getSession(), "test_varchar")) + .execute(getQueryRunner(), trinoCreateAsSelect("test_varchar")); + } + + @Test + public void testSqlServerNvarchar() + { + // Unicode literals must be prefixed with N'string' + // https://docs.microsoft.com/en-us/sql/analytics-platform-system/load-with-insert?view=aps-pdw-2016-au7#char-varchar-nchar-and-nvarchar-data-types + SqlDataTypeTest.create() + .addRoundTrip("nvarchar(5)", "N'攻殻機動隊'", createVarcharType(5), "CAST('攻殻機動隊' AS varchar(5))") + .addRoundTrip("nvarchar(32)", "N'攻殻機動隊'", createVarcharType(32), "CAST('攻殻機動隊' AS varchar(32))") + .addRoundTrip("nvarchar(20)", "N'😂'", createVarcharType(20), "CAST('😂' AS varchar(20))") + .addRoundTrip("nvarchar(77)", "N'Ну, погоди!'", createVarcharType(77), "CAST('Ну, погоди!' AS varchar(77))") + .execute(getQueryRunner(), sqlServerCreateAndInsert("test_sqlserver_nvarchar")); + } + + @Test + public void testTrinoLongVarchar() + { + // testing mapping varchar > 4000 -> varchar(max) + SqlDataTypeTest.create() + .addRoundTrip("varchar(4001)", "'text_c'", createUnboundedVarcharType(), "CAST('text_c' AS varchar)") + .execute(getQueryRunner(), trinoCreateAndInsert(getSession(), "test_long_varchar")) + .execute(getQueryRunner(), trinoCreateAsSelect("test_long_varchar")); + } + + @Test + public void testTrinoUnboundedVarchar() + { + SqlDataTypeTest.create() + .addRoundTrip("varchar", "'text_a'", createUnboundedVarcharType(), "CAST('text_a' AS varchar)") + .addRoundTrip("varchar", "'text_b'", createUnboundedVarcharType(), "CAST('text_b' AS varchar)") + .addRoundTrip("varchar", "'text_d'", createUnboundedVarcharType(), "CAST('text_d' AS varchar)") + .addRoundTrip("varchar", "CAST('攻殻機動隊' AS varchar)", createUnboundedVarcharType(), "CAST('攻殻機動隊' AS varchar)") + .addRoundTrip("varchar", "CAST('攻殻機動隊' AS varchar)", createUnboundedVarcharType(), "CAST('攻殻機動隊' AS varchar)") + .addRoundTrip("varchar", "CAST('攻殻機動隊' AS varchar)", createUnboundedVarcharType(), "CAST('攻殻機動隊' AS varchar)") + .addRoundTrip("varchar", "CAST('😂' AS varchar)", createUnboundedVarcharType(), "CAST('😂' AS varchar)") + .addRoundTrip("varchar", "CAST('Ну, погоди!' AS varchar)", createUnboundedVarcharType(), "CAST('Ну, погоди!' AS varchar)") + .addRoundTrip("varchar", "'text_f'", createUnboundedVarcharType(), "CAST('text_f' AS varchar)") + .execute(getQueryRunner(), trinoCreateAndInsert(getSession(), "test_unbounded_varchar")) + .execute(getQueryRunner(), trinoCreateAsSelect("test_unbounded_varchar")); + } + + @Test + public void testVarbinary() + { + SqlDataTypeTest.create() + .addRoundTrip("varbinary", "NULL", VARBINARY, "CAST(NULL AS varbinary)") + .addRoundTrip("varbinary", "X''", VARBINARY, "X''") + .addRoundTrip("varbinary", "X'68656C6C6F'", VARBINARY, "to_utf8('hello')") + .addRoundTrip("varbinary", "X'5069C4996B6E6120C582C4856B61207720E69DB1E4BAACE983BD'", VARBINARY, "to_utf8('Piękna łąka w 東京都')") + .addRoundTrip("varbinary", "X'4261672066756C6C206F6620F09F92B0'", VARBINARY, "to_utf8('Bag full of 💰')") + .addRoundTrip("varbinary", "X'0001020304050607080DF9367AA7000000'", VARBINARY, "X'0001020304050607080DF9367AA7000000'") // non-text + .addRoundTrip("varbinary", "X'000000000000'", VARBINARY, "X'000000000000'") + .execute(getQueryRunner(), trinoCreateAsSelect("test_varbinary")); + + // Binary literals must be prefixed with 0x + // https://docs.microsoft.com/en-us/sql/analytics-platform-system/load-with-insert?view=aps-pdw-2016-au7#InsertingLiteralsBinary + SqlDataTypeTest.create() + .addRoundTrip("varbinary(10)", "NULL", VARBINARY, "CAST(NULL AS varbinary)") + .addRoundTrip("varbinary(20)", "0x", VARBINARY, "CAST('' AS varbinary)") + .addRoundTrip("varbinary(30)", "0x68656C6C6F", VARBINARY, "X'68656C6C6F'") + .addRoundTrip("varbinary(1000)", "0x5069C4996B6E6120C582C4856B61207720E69DB1E4BAACE983BD", VARBINARY, "X'5069C4996B6E6120C582C4856B61207720E69DB1E4BAACE983BD'") + .addRoundTrip("varbinary(2000)", "0x4261672066756C6C206F6620F09F92B0", VARBINARY, "X'4261672066756C6C206F6620F09F92B0'") + .addRoundTrip("varbinary(4000)", "0x0001020304050607080DF9367AA7000000", VARBINARY, "X'0001020304050607080DF9367AA7000000'") // non-text + .addRoundTrip("varbinary(4000)", "0x000000000000", VARBINARY, "X'000000000000'") + .execute(getQueryRunner(), sqlServerCreateAndInsert("test_varbinary")); + } + + @Test + public void testDate() + { + ZoneId jvmZone = ZoneId.systemDefault(); + checkState(jvmZone.getId().equals("America/Bahia_Banderas"), "This test assumes certain JVM time zone"); + LocalDate dateOfLocalTimeChangeForwardAtMidnightInJvmZone = LocalDate.of(1970, 1, 1); + checkIsGap(jvmZone, dateOfLocalTimeChangeForwardAtMidnightInJvmZone.atStartOfDay()); + + ZoneId someZone = ZoneId.of("Europe/Vilnius"); + LocalDate dateOfLocalTimeChangeForwardAtMidnightInSomeZone = LocalDate.of(1983, 4, 1); + checkIsGap(someZone, dateOfLocalTimeChangeForwardAtMidnightInSomeZone.atStartOfDay()); + LocalDate dateOfLocalTimeChangeBackwardAtMidnightInSomeZone = LocalDate.of(1983, 10, 1); + checkIsDoubled(someZone, dateOfLocalTimeChangeBackwardAtMidnightInSomeZone.atStartOfDay().minusMinutes(1)); + + SqlDataTypeTest testsSqlServer = SqlDataTypeTest.create() + .addRoundTrip("date", "NULL", DATE, "CAST(NULL AS DATE)") + // before epoch + .addRoundTrip("date", "'1952-04-03'", DATE, "DATE '1952-04-03'") + .addRoundTrip("date", "'1970-01-01'", DATE, "DATE '1970-01-01'") + .addRoundTrip("date", "'1970-02-03'", DATE, "DATE '1970-02-03'") + // summer on northern hemisphere (possible DST) + .addRoundTrip("date", "'2017-07-01'", DATE, "DATE '2017-07-01'") + // winter on northern hemisphere (possible DST on southern hemisphere) + .addRoundTrip("date", "'2017-01-01'", DATE, "DATE '2017-01-01'") + .addRoundTrip("date", "'1970-01-01'", DATE, "DATE '1970-01-01'") + .addRoundTrip("date", "'1983-04-01'", DATE, "DATE '1983-04-01'") + .addRoundTrip("date", "'1983-10-01'", DATE, "DATE '1983-10-01'"); + + SqlDataTypeTest testsTrino = SqlDataTypeTest.create() + .addRoundTrip("date", "NULL", DATE, "CAST(NULL AS DATE)") + // before epoch + .addRoundTrip("date", "DATE '1952-04-03'", DATE, "DATE '1952-04-03'") + .addRoundTrip("date", "DATE '1970-01-01'", DATE, "DATE '1970-01-01'") + .addRoundTrip("date", "DATE '1970-02-03'", DATE, "DATE '1970-02-03'") + // summer on northern hemisphere (possible DST) + .addRoundTrip("date", "DATE '2017-07-01'", DATE, "DATE '2017-07-01'") + // winter on northern hemisphere (possible DST on southern hemisphere) + .addRoundTrip("date", "DATE '2017-01-01'", DATE, "DATE '2017-01-01'") + .addRoundTrip("date", "DATE '1970-01-01'", DATE, "DATE '1970-01-01'") + .addRoundTrip("date", "DATE '1983-04-01'", DATE, "DATE '1983-04-01'") + .addRoundTrip("date", "DATE '1983-10-01'", DATE, "DATE '1983-10-01'"); + + for (String timeZoneId : ImmutableList.of(UTC_KEY.getId(), jvmZone.getId(), someZone.getId())) { + Session session = Session.builder(getSession()) + .setTimeZoneKey(TimeZoneKey.getTimeZoneKey(timeZoneId)) + .build(); + testsSqlServer.execute(getQueryRunner(), session, sqlServerCreateAndInsert("test_date")); + testsTrino.execute(getQueryRunner(), session, trinoCreateAsSelect(session, "test_date")); + testsTrino.execute(getQueryRunner(), session, trinoCreateAndInsert(session, "test_date")); + } + } + + @Test + public void testTime() + { + SqlDataTypeTest.create() + .addRoundTrip("time(0)", "'00:00:00'", createTimeType(0), "TIME '00:00:00'") + .addRoundTrip("time(6)", "'00:00:00.000000'", createTimeType(6), "TIME '00:00:00.000000'") + .addRoundTrip("time(6)", "'00:00:00.123456'", createTimeType(6), "TIME '00:00:00.123456'") + .addRoundTrip("time(0)", "'12:34:56'", createTimeType(0), "TIME '12:34:56'") + .addRoundTrip("time(6)", "'12:34:56.123456'", createTimeType(6), "TIME '12:34:56.123456'") + + // maximal value for a precision + .addRoundTrip("time(0)", "'23:59:59'", createTimeType(0), "TIME '23:59:59'") + .addRoundTrip("time(1)", "'23:59:59.9'", createTimeType(1), "TIME '23:59:59.9'") + .addRoundTrip("time(2)", "'23:59:59.99'", createTimeType(2), "TIME '23:59:59.99'") + .addRoundTrip("time(3)", "'23:59:59.999'", createTimeType(3), "TIME '23:59:59.999'") + .addRoundTrip("time(4)", "'23:59:59.9999'", createTimeType(4), "TIME '23:59:59.9999'") + .addRoundTrip("time(5)", "'23:59:59.99999'", createTimeType(5), "TIME '23:59:59.99999'") + .addRoundTrip("time(6)", "'23:59:59.999999'", createTimeType(6), "TIME '23:59:59.999999'") + .addRoundTrip("time(7)", "'23:59:59.9999999'", createTimeType(7), "TIME '23:59:59.9999999'") + + .execute(getQueryRunner(), sqlServerCreateAndInsert("test_time")); + + SqlDataTypeTest.create() + .addRoundTrip("TIME '00:00:00'", "TIME '00:00:00'") + .addRoundTrip("TIME '00:00:00.000000'", "TIME '00:00:00.000000'") + .addRoundTrip("TIME '00:00:00.123456'", "TIME '00:00:00.123456'") + .addRoundTrip("TIME '12:34:56'", "TIME '12:34:56'") + .addRoundTrip("TIME '12:34:56.123456'", "TIME '12:34:56.123456'") + + // maximal value for a precision + .addRoundTrip("TIME '23:59:59'", "TIME '23:59:59'") + .addRoundTrip("TIME '23:59:59.9'", "TIME '23:59:59.9'") + .addRoundTrip("TIME '23:59:59.99'", "TIME '23:59:59.99'") + .addRoundTrip("TIME '23:59:59.999'", "TIME '23:59:59.999'") + .addRoundTrip("TIME '23:59:59.9999'", "TIME '23:59:59.9999'") + .addRoundTrip("TIME '23:59:59.99999'", "TIME '23:59:59.99999'") + .addRoundTrip("TIME '23:59:59.999999'", "TIME '23:59:59.999999'") + .addRoundTrip("TIME '23:59:59.9999999'", "TIME '23:59:59.9999999'") + + .execute(getQueryRunner(), trinoCreateAsSelect(getSession(), "test_time")) + .execute(getQueryRunner(), trinoCreateAndInsert(getSession(), "test_time")); + + SqlDataTypeTest.create() + // round down + .addRoundTrip("TIME '00:00:00.00000001'", "TIME '00:00:00.0000000'") + .addRoundTrip("TIME '00:00:00.000000000001'", "TIME '00:00:00.0000000'") + + // round down, maximal value + .addRoundTrip("TIME '00:00:00.00000004'", "TIME '00:00:00.0000000'") + .addRoundTrip("TIME '00:00:00.000000049'", "TIME '00:00:00.0000000'") + .addRoundTrip("TIME '00:00:00.0000000449'", "TIME '00:00:00.0000000'") + .addRoundTrip("TIME '00:00:00.00000004449'", "TIME '00:00:00.0000000'") + .addRoundTrip("TIME '00:00:00.000000044449'", "TIME '00:00:00.0000000'") + + // round up to next day, minimal value + .addRoundTrip("TIME '23:59:59.99999995'", "TIME '00:00:00.0000000'") + .addRoundTrip("TIME '23:59:59.999999950'", "TIME '00:00:00.0000000'") + .addRoundTrip("TIME '23:59:59.9999999500'", "TIME '00:00:00.0000000'") + .addRoundTrip("TIME '23:59:59.99999995000'", "TIME '00:00:00.0000000'") + .addRoundTrip("TIME '23:59:59.999999950000'", "TIME '00:00:00.0000000'") + + // round up to next day, maximal value + .addRoundTrip("TIME '23:59:59.99999999'", "TIME '00:00:00.0000000'") + .addRoundTrip("TIME '23:59:59.999999999'", "TIME '00:00:00.0000000'") + .addRoundTrip("TIME '23:59:59.9999999999'", "TIME '00:00:00.0000000'") + .addRoundTrip("TIME '23:59:59.99999999999'", "TIME '00:00:00.0000000'") + .addRoundTrip("TIME '23:59:59.999999999999'", "TIME '00:00:00.0000000'") + + // round down + .addRoundTrip("TIME '23:59:59.999999949999'", "TIME '23:59:59.9999999'") + + .execute(getQueryRunner(), trinoCreateAndInsert(getSession(), "test_time")) + .execute(getQueryRunner(), trinoCreateAsSelect(getSession(), "test_time")); + } + + @Test(dataProvider = "testTimestampDataProvider") + public void testTimestamp(ZoneId sessionZone) + { + SqlDataTypeTest tests = SqlDataTypeTest.create() + + // before epoch + .addRoundTrip("TIMESTAMP '1958-01-01 13:18:03.123'", "TIMESTAMP '1958-01-01 13:18:03.123'") + // after epoch + .addRoundTrip("TIMESTAMP '2019-03-18 10:01:17.987'", "TIMESTAMP '2019-03-18 10:01:17.987'") + // time doubled in JVM zone + .addRoundTrip("TIMESTAMP '2018-10-28 01:33:17.456'", "TIMESTAMP '2018-10-28 01:33:17.456'") + // time double in Vilnius + .addRoundTrip("TIMESTAMP '2018-10-28 03:33:33.333'", "TIMESTAMP '2018-10-28 03:33:33.333'") + // epoch + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.000'", "TIMESTAMP '1970-01-01 00:00:00.000'") + // time gap in JVM zone + .addRoundTrip("TIMESTAMP '1970-01-01 00:13:42.000'", "TIMESTAMP '1970-01-01 00:13:42.000'") + .addRoundTrip("TIMESTAMP '2018-04-01 02:13:55.123'", "TIMESTAMP '2018-04-01 02:13:55.123'") + // time gap in Vilnius + .addRoundTrip("TIMESTAMP '2018-03-25 03:17:17.000'", "TIMESTAMP '2018-03-25 03:17:17.000'") + // time gap in Kathmandu + .addRoundTrip("TIMESTAMP '1986-01-01 00:13:07.000'", "TIMESTAMP '1986-01-01 00:13:07.000'") + + // same as above but with higher precision + .addRoundTrip("TIMESTAMP '1958-01-01 13:18:03.1230000'", "TIMESTAMP '1958-01-01 13:18:03.1230000'") + .addRoundTrip("TIMESTAMP '2019-03-18 10:01:17.9870000'", "TIMESTAMP '2019-03-18 10:01:17.9870000'") + .addRoundTrip("TIMESTAMP '2018-10-28 01:33:17.4560000'", "TIMESTAMP '2018-10-28 01:33:17.4560000'") + .addRoundTrip("TIMESTAMP '2018-10-28 03:33:33.3330000'", "TIMESTAMP '2018-10-28 03:33:33.3330000'") + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.0000000'", "TIMESTAMP '1970-01-01 00:00:00.0000000'") + .addRoundTrip("TIMESTAMP '1970-01-01 00:13:42.0000000'", "TIMESTAMP '1970-01-01 00:13:42.0000000'") + .addRoundTrip("TIMESTAMP '2018-04-01 02:13:55.1230000'", "TIMESTAMP '2018-04-01 02:13:55.1230000'") + .addRoundTrip("TIMESTAMP '2018-03-25 03:17:17.0000000'", "TIMESTAMP '2018-03-25 03:17:17.0000000'") + .addRoundTrip("TIMESTAMP '1986-01-01 00:13:07.0000000'", "TIMESTAMP '1986-01-01 00:13:07.0000000'") + + // test arbitrary time for all supported precisions + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00'", "TIMESTAMP '1970-01-01 00:00:00'") + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.1'", "TIMESTAMP '1970-01-01 00:00:00.1'") + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.12'", "TIMESTAMP '1970-01-01 00:00:00.12'") + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.123'", "TIMESTAMP '1970-01-01 00:00:00.123'") + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.1234'", "TIMESTAMP '1970-01-01 00:00:00.1234'") + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.12345'", "TIMESTAMP '1970-01-01 00:00:00.12345'") + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.123456'", "TIMESTAMP '1970-01-01 00:00:00.123456'") + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.1234567'", "TIMESTAMP '1970-01-01 00:00:00.1234567'") + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.12345670'", "TIMESTAMP '1970-01-01 00:00:00.1234567'") + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.123456749999'", "TIMESTAMP '1970-01-01 00:00:00.1234567'") + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.12345675'", "TIMESTAMP '1970-01-01 00:00:00.1234568'") + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.12345679'", "TIMESTAMP '1970-01-01 00:00:00.1234568'") + + // before epoch with second fraction + .addRoundTrip("TIMESTAMP '1969-12-31 23:59:59.1230000'", "TIMESTAMP '1969-12-31 23:59:59.1230000'") + .addRoundTrip("TIMESTAMP '1969-12-31 23:59:59.1234567'", "TIMESTAMP '1969-12-31 23:59:59.1234567'") + + // precision 0 ends up as precision 0 + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00'", "TIMESTAMP '1970-01-01 00:00:00'") + + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.1'", "TIMESTAMP '1970-01-01 00:00:00.1'") + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.9'", "TIMESTAMP '1970-01-01 00:00:00.9'") + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.123'", "TIMESTAMP '1970-01-01 00:00:00.123'") + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.123000'", "TIMESTAMP '1970-01-01 00:00:00.123000'") + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.999'", "TIMESTAMP '1970-01-01 00:00:00.999'") + // max supported precision + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.1234567'", "TIMESTAMP '1970-01-01 00:00:00.1234567'") + + .addRoundTrip("TIMESTAMP '2020-09-27 12:34:56.1'", "TIMESTAMP '2020-09-27 12:34:56.1'") + .addRoundTrip("TIMESTAMP '2020-09-27 12:34:56.9'", "TIMESTAMP '2020-09-27 12:34:56.9'") + .addRoundTrip("TIMESTAMP '2020-09-27 12:34:56.123'", "TIMESTAMP '2020-09-27 12:34:56.123'") + .addRoundTrip("TIMESTAMP '2020-09-27 12:34:56.123000'", "TIMESTAMP '2020-09-27 12:34:56.123000'") + .addRoundTrip("TIMESTAMP '2020-09-27 12:34:56.999'", "TIMESTAMP '2020-09-27 12:34:56.999'") + // max supported precision + .addRoundTrip("TIMESTAMP '2020-09-27 12:34:56.1234567'", "TIMESTAMP '2020-09-27 12:34:56.1234567'") + + // round down + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.12345671'", "TIMESTAMP '1970-01-01 00:00:00.1234567'") + + // nanos round up, end result rounds down + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.1234567499'", "TIMESTAMP '1970-01-01 00:00:00.1234567'") + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.123456749999'", "TIMESTAMP '1970-01-01 00:00:00.1234567'") + + // round up + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.12345675'", "TIMESTAMP '1970-01-01 00:00:00.1234568'") + + // max precision + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.111222333444'", "TIMESTAMP '1970-01-01 00:00:00.1112223'") + + // round up to next second + .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.99999995'", "TIMESTAMP '1970-01-01 00:00:01.0000000'") + + // round up to next day + .addRoundTrip("TIMESTAMP '1970-01-01 23:59:59.99999995'", "TIMESTAMP '1970-01-02 00:00:00.0000000'") + + // negative epoch + .addRoundTrip("TIMESTAMP '1969-12-31 23:59:59.99999995'", "TIMESTAMP '1970-01-01 00:00:00.0000000'") + .addRoundTrip("TIMESTAMP '1969-12-31 23:59:59.999999949999'", "TIMESTAMP '1969-12-31 23:59:59.9999999'") + .addRoundTrip("TIMESTAMP '1969-12-31 23:59:59.99999994'", "TIMESTAMP '1969-12-31 23:59:59.9999999'"); + + Session session = Session.builder(getSession()) + .setTimeZoneKey(TimeZoneKey.getTimeZoneKey(sessionZone.getId())) + .build(); + + tests.execute(getQueryRunner(), session, trinoCreateAsSelect(session, "test_timestamp")); + tests.execute(getQueryRunner(), session, trinoCreateAsSelect(getSession(), "test_timestamp")); + tests.execute(getQueryRunner(), session, trinoCreateAndInsert(session, "test_timestamp")); + } + + @Test + public void testSqlServerDatetime2() + { + SqlDataTypeTest.create() + // literal values with higher precision are NOT rounded and cause an error + .addRoundTrip("DATETIME2(0)", "'1970-01-01 00:00:00'", createTimestampType(0), "TIMESTAMP '1970-01-01 00:00:00'") + .addRoundTrip("DATETIME2(1)", "'1970-01-01 00:00:00.1'", createTimestampType(1), "TIMESTAMP '1970-01-01 00:00:00.1'") + .addRoundTrip("DATETIME2(1)", "'1970-01-01 00:00:00.9'", createTimestampType(1), "TIMESTAMP '1970-01-01 00:00:00.9'") + .addRoundTrip("DATETIME2(3)", "'1970-01-01 00:00:00.123'", createTimestampType(3), "TIMESTAMP '1970-01-01 00:00:00.123'") + .addRoundTrip("DATETIME2(6)", "'1970-01-01 00:00:00.123000'", createTimestampType(6), "TIMESTAMP '1970-01-01 00:00:00.123000'") + .addRoundTrip("DATETIME2(3)", "'1970-01-01 00:00:00.999'", createTimestampType(3), "TIMESTAMP '1970-01-01 00:00:00.999'") + .addRoundTrip("DATETIME2(7)", "'1970-01-01 00:00:00.1234567'", createTimestampType(7), "TIMESTAMP '1970-01-01 00:00:00.1234567'") + .addRoundTrip("DATETIME2(1)", "'2020-09-27 12:34:56.1'", createTimestampType(1), "TIMESTAMP '2020-09-27 12:34:56.1'") + .addRoundTrip("DATETIME2(1)", "'2020-09-27 12:34:56.9'", createTimestampType(1), "TIMESTAMP '2020-09-27 12:34:56.9'") + .addRoundTrip("DATETIME2(3)", "'2020-09-27 12:34:56.123'", createTimestampType(3), "TIMESTAMP '2020-09-27 12:34:56.123'") + .addRoundTrip("DATETIME2(6)", "'2020-09-27 12:34:56.123000'", createTimestampType(6), "TIMESTAMP '2020-09-27 12:34:56.123000'") + .addRoundTrip("DATETIME2(3)", "'2020-09-27 12:34:56.999'", createTimestampType(3), "TIMESTAMP '2020-09-27 12:34:56.999'") + .addRoundTrip("DATETIME2(7)", "'2020-09-27 12:34:56.1234567'", createTimestampType(7), "TIMESTAMP '2020-09-27 12:34:56.1234567'") + + .addRoundTrip("DATETIME2(7)", "'1970-01-01 00:00:00'", createTimestampType(7), "TIMESTAMP '1970-01-01 00:00:00.0000000'") + .addRoundTrip("DATETIME2(7)", "'1970-01-01 00:00:00.1'", createTimestampType(7), "TIMESTAMP '1970-01-01 00:00:00.1000000'") + .addRoundTrip("DATETIME2(7)", "'1970-01-01 00:00:00.9'", createTimestampType(7), "TIMESTAMP '1970-01-01 00:00:00.9000000'") + .addRoundTrip("DATETIME2(7)", "'1970-01-01 00:00:00.123'", createTimestampType(7), "TIMESTAMP '1970-01-01 00:00:00.1230000'") + .addRoundTrip("DATETIME2(7)", "'1970-01-01 00:00:00.123000'", createTimestampType(7), "TIMESTAMP '1970-01-01 00:00:00.1230000'") + .addRoundTrip("DATETIME2(7)", "'1970-01-01 00:00:00.999'", createTimestampType(7), "TIMESTAMP '1970-01-01 00:00:00.9990000'") + .addRoundTrip("DATETIME2(7)", "'1970-01-01 00:00:00.1234567'", createTimestampType(7), "TIMESTAMP '1970-01-01 00:00:00.1234567'") + .addRoundTrip("DATETIME2(7)", "'2020-09-27 12:34:56.1'", createTimestampType(7), "TIMESTAMP '2020-09-27 12:34:56.1000000'") + .addRoundTrip("DATETIME2(7)", "'2020-09-27 12:34:56.9'", createTimestampType(7), "TIMESTAMP '2020-09-27 12:34:56.9000000'") + .addRoundTrip("DATETIME2(7)", "'2020-09-27 12:34:56.123'", createTimestampType(7), "TIMESTAMP '2020-09-27 12:34:56.1230000'") + .addRoundTrip("DATETIME2(7)", "'2020-09-27 12:34:56.123000'", createTimestampType(7), "TIMESTAMP '2020-09-27 12:34:56.1230000'") + .addRoundTrip("DATETIME2(7)", "'2020-09-27 12:34:56.999'", createTimestampType(7), "TIMESTAMP '2020-09-27 12:34:56.9990000'") + .addRoundTrip("DATETIME2(7)", "'2020-09-27 12:34:56.1234567'", createTimestampType(7), "TIMESTAMP '2020-09-27 12:34:56.1234567'") + + .execute(getQueryRunner(), sqlServerCreateAndInsert("test_sqlserver_timestamp")); + } + + @DataProvider + public Object[][] testTimestampDataProvider() + { + return new Object[][] { + {UTC}, + {ZoneId.systemDefault()}, + // using two non-JVM zones so that we don't need to worry what SQL Server system zone is + // no DST in 1970, but has DST in later years (e.g. 2018) + {ZoneId.of("Europe/Vilnius")}, + // minutes offset change since 1970-01-01, no DST + {ZoneId.of("Asia/Kathmandu")}, + {ZoneId.of(TestingSession.DEFAULT_TIME_ZONE_KEY.getId())}, + }; + } + + protected DataSetup trinoCreateAsSelect(String tableNamePrefix) + { + return trinoCreateAsSelect(getSession(), tableNamePrefix); + } + + protected DataSetup trinoCreateAsSelect(Session session, String tableNamePrefix) + { + return new CreateAsSelectDataSetup(new TrinoSqlExecutor(getQueryRunner(), session), tableNamePrefix); + } + + protected DataSetup trinoCreateAndInsert(Session session, String tableNamePrefix) + { + return new CreateAndInsertDataSetup(new TrinoSqlExecutor(getQueryRunner(), session), tableNamePrefix); + } + + protected DataSetup sqlServerCreateAndInsert(String tableNamePrefix) + { + return new CreateAndInsertDataSetup(onRemoteDatabase(), tableNamePrefix); + } + + private static void checkIsDoubled(ZoneId zone, LocalDateTime dateTime) + { + verify(zone.getRules().getValidOffsets(dateTime).size() == 2, "Expected %s to be doubled in %s", dateTime, zone); + } + + private static boolean isGap(ZoneId zone, LocalDateTime dateTime) + { + return zone.getRules().getValidOffsets(dateTime).isEmpty(); + } + + private static void checkIsGap(ZoneId zone, LocalDateTime dateTime) + { + verify(isGap(zone, dateTime), "Expected %s to be a gap in %s", dateTime, zone); + } + + protected abstract SqlExecutor onRemoteDatabase(); +} diff --git a/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/TestSqlServerClient.java b/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/TestSqlServerClient.java index c7f0bb951ebf..c5519986fc9e 100644 --- a/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/TestSqlServerClient.java +++ b/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/TestSqlServerClient.java @@ -56,6 +56,7 @@ public class TestSqlServerClient private static final JdbcClient JDBC_CLIENT = new SqlServerClient( new BaseJdbcConfig(), + new SqlServerConfig(), session -> { throw new UnsupportedOperationException(); }); diff --git a/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/TestSqlServerConfig.java b/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/TestSqlServerConfig.java new file mode 100644 index 000000000000..c2e5b1750613 --- /dev/null +++ b/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/TestSqlServerConfig.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 io.trino.plugin.sqlserver; + +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 TestSqlServerConfig +{ + @Test + public void testDefaults() + { + assertRecordedDefaults(recordDefaults(SqlServerConfig.class) + .setSnapshotIsolationDisabled(false)); + } + + @Test + public void testExplicitPropertyMappings() + { + Map properties = new ImmutableMap.Builder() + .put("sqlserver.snapshot-isolation.disabled", "true") + .build(); + + SqlServerConfig expected = new SqlServerConfig() + .setSnapshotIsolationDisabled(true); + + assertFullMapping(properties, expected); + } +} diff --git a/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/TestSqlServerTypeMapping.java b/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/TestSqlServerTypeMapping.java index 8b1d3bee41ef..6a9c8cbd67d6 100644 --- a/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/TestSqlServerTypeMapping.java +++ b/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/TestSqlServerTypeMapping.java @@ -15,33 +15,15 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import io.trino.Session; -import io.trino.spi.type.TimeZoneKey; -import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.QueryRunner; -import io.trino.testing.TestingSession; -import io.trino.testing.datatype.CreateAndInsertDataSetup; -import io.trino.testing.datatype.CreateAsSelectDataSetup; -import io.trino.testing.datatype.DataSetup; -import io.trino.testing.datatype.SqlDataTypeTest; -import io.trino.testing.sql.JdbcSqlExecutor; -import io.trino.testing.sql.TrinoSqlExecutor; -import org.testng.annotations.DataProvider; -import org.testng.annotations.Test; - -import java.time.ZoneId; -import java.util.Properties; +import io.trino.testing.sql.SqlExecutor; import static io.trino.plugin.sqlserver.SqlServerQueryRunner.createSqlServerQueryRunner; -import static io.trino.spi.type.TimeType.createTimeType; -import static io.trino.spi.type.TimestampType.createTimestampType; -import static io.trino.spi.type.VarbinaryType.VARBINARY; -import static java.time.ZoneOffset.UTC; public class TestSqlServerTypeMapping - extends AbstractTestQueryFramework + extends BaseSqlServerTypeMapping { - private TestingSqlServer sqlServer; + protected TestingSqlServer sqlServer; @Override protected QueryRunner createQueryRunner() @@ -56,270 +38,9 @@ protected QueryRunner createQueryRunner() ImmutableList.of()); } - @Test - public void testVarbinary() - { - SqlDataTypeTest.create() - .addRoundTrip("varbinary", "NULL", VARBINARY, "CAST(NULL AS varbinary)") - .addRoundTrip("varbinary", "X''", VARBINARY, "X''") - .addRoundTrip("varbinary", "X'68656C6C6F'", VARBINARY, "to_utf8('hello')") - .addRoundTrip("varbinary", "X'5069C4996B6E6120C582C4856B61207720E69DB1E4BAACE983BD'", VARBINARY, "to_utf8('Piękna łąka w 東京都')") - .addRoundTrip("varbinary", "X'4261672066756C6C206F6620F09F92B0'", VARBINARY, "to_utf8('Bag full of 💰')") - .addRoundTrip("varbinary", "X'0001020304050607080DF9367AA7000000'", VARBINARY, "X'0001020304050607080DF9367AA7000000'") // non-text - .addRoundTrip("varbinary", "X'000000000000'", VARBINARY, "X'000000000000'") - .execute(getQueryRunner(), trinoCreateAsSelect("test_varbinary")); - } - - @Test - public void testTime() - { - SqlDataTypeTest.create() - .addRoundTrip("time(0)", "'00:00:00'", createTimeType(0), "TIME '00:00:00'") - .addRoundTrip("time(6)", "'00:00:00.000000'", createTimeType(6), "TIME '00:00:00.000000'") - .addRoundTrip("time(6)", "'00:00:00.123456'", createTimeType(6), "TIME '00:00:00.123456'") - .addRoundTrip("time(0)", "'12:34:56'", createTimeType(0), "TIME '12:34:56'") - .addRoundTrip("time(6)", "'12:34:56.123456'", createTimeType(6), "TIME '12:34:56.123456'") - - // maximal value for a precision - .addRoundTrip("time(0)", "'23:59:59'", createTimeType(0), "TIME '23:59:59'") - .addRoundTrip("time(1)", "'23:59:59.9'", createTimeType(1), "TIME '23:59:59.9'") - .addRoundTrip("time(2)", "'23:59:59.99'", createTimeType(2), "TIME '23:59:59.99'") - .addRoundTrip("time(3)", "'23:59:59.999'", createTimeType(3), "TIME '23:59:59.999'") - .addRoundTrip("time(4)", "'23:59:59.9999'", createTimeType(4), "TIME '23:59:59.9999'") - .addRoundTrip("time(5)", "'23:59:59.99999'", createTimeType(5), "TIME '23:59:59.99999'") - .addRoundTrip("time(6)", "'23:59:59.999999'", createTimeType(6), "TIME '23:59:59.999999'") - .addRoundTrip("time(7)", "'23:59:59.9999999'", createTimeType(7), "TIME '23:59:59.9999999'") - - .execute(getQueryRunner(), sqlServerCreateAndInsert("test_time")); - - SqlDataTypeTest.create() - .addRoundTrip("TIME '00:00:00'", "TIME '00:00:00'") - .addRoundTrip("TIME '00:00:00.000000'", "TIME '00:00:00.000000'") - .addRoundTrip("TIME '00:00:00.123456'", "TIME '00:00:00.123456'") - .addRoundTrip("TIME '12:34:56'", "TIME '12:34:56'") - .addRoundTrip("TIME '12:34:56.123456'", "TIME '12:34:56.123456'") - - // maximal value for a precision - .addRoundTrip("TIME '23:59:59'", "TIME '23:59:59'") - .addRoundTrip("TIME '23:59:59.9'", "TIME '23:59:59.9'") - .addRoundTrip("TIME '23:59:59.99'", "TIME '23:59:59.99'") - .addRoundTrip("TIME '23:59:59.999'", "TIME '23:59:59.999'") - .addRoundTrip("TIME '23:59:59.9999'", "TIME '23:59:59.9999'") - .addRoundTrip("TIME '23:59:59.99999'", "TIME '23:59:59.99999'") - .addRoundTrip("TIME '23:59:59.999999'", "TIME '23:59:59.999999'") - .addRoundTrip("TIME '23:59:59.9999999'", "TIME '23:59:59.9999999'") - - .execute(getQueryRunner(), trinoCreateAsSelect(getSession(), "test_time")) - .execute(getQueryRunner(), trinoCreateAndInsert(getSession(), "test_time")); - - SqlDataTypeTest.create() - // round down - .addRoundTrip("TIME '00:00:00.00000001'", "TIME '00:00:00.0000000'") - .addRoundTrip("TIME '00:00:00.000000000001'", "TIME '00:00:00.0000000'") - - // round down, maximal value - .addRoundTrip("TIME '00:00:00.00000004'", "TIME '00:00:00.0000000'") - .addRoundTrip("TIME '00:00:00.000000049'", "TIME '00:00:00.0000000'") - .addRoundTrip("TIME '00:00:00.0000000449'", "TIME '00:00:00.0000000'") - .addRoundTrip("TIME '00:00:00.00000004449'", "TIME '00:00:00.0000000'") - .addRoundTrip("TIME '00:00:00.000000044449'", "TIME '00:00:00.0000000'") - - // round up to next day, minimal value - .addRoundTrip("TIME '23:59:59.99999995'", "TIME '00:00:00.0000000'") - .addRoundTrip("TIME '23:59:59.999999950'", "TIME '00:00:00.0000000'") - .addRoundTrip("TIME '23:59:59.9999999500'", "TIME '00:00:00.0000000'") - .addRoundTrip("TIME '23:59:59.99999995000'", "TIME '00:00:00.0000000'") - .addRoundTrip("TIME '23:59:59.999999950000'", "TIME '00:00:00.0000000'") - - // round up to next day, maximal value - .addRoundTrip("TIME '23:59:59.99999999'", "TIME '00:00:00.0000000'") - .addRoundTrip("TIME '23:59:59.999999999'", "TIME '00:00:00.0000000'") - .addRoundTrip("TIME '23:59:59.9999999999'", "TIME '00:00:00.0000000'") - .addRoundTrip("TIME '23:59:59.99999999999'", "TIME '00:00:00.0000000'") - .addRoundTrip("TIME '23:59:59.999999999999'", "TIME '00:00:00.0000000'") - - // round down - .addRoundTrip("TIME '23:59:59.999999949999'", "TIME '23:59:59.9999999'") - - .execute(getQueryRunner(), trinoCreateAndInsert(getSession(), "test_time")) - .execute(getQueryRunner(), trinoCreateAsSelect(getSession(), "test_time")); - } - - @Test(dataProvider = "testTimestampDataProvider") - public void testTimestamp(ZoneId sessionZone) - { - SqlDataTypeTest tests = SqlDataTypeTest.create() - - // before epoch - .addRoundTrip("TIMESTAMP '1958-01-01 13:18:03.123'", "TIMESTAMP '1958-01-01 13:18:03.123'") - // after epoch - .addRoundTrip("TIMESTAMP '2019-03-18 10:01:17.987'", "TIMESTAMP '2019-03-18 10:01:17.987'") - // time doubled in JVM zone - .addRoundTrip("TIMESTAMP '2018-10-28 01:33:17.456'", "TIMESTAMP '2018-10-28 01:33:17.456'") - // time double in Vilnius - .addRoundTrip("TIMESTAMP '2018-10-28 03:33:33.333'", "TIMESTAMP '2018-10-28 03:33:33.333'") - // epoch - .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.000'", "TIMESTAMP '1970-01-01 00:00:00.000'") - // time gap in JVM zone - .addRoundTrip("TIMESTAMP '1970-01-01 00:13:42.000'", "TIMESTAMP '1970-01-01 00:13:42.000'") - .addRoundTrip("TIMESTAMP '2018-04-01 02:13:55.123'", "TIMESTAMP '2018-04-01 02:13:55.123'") - // time gap in Vilnius - .addRoundTrip("TIMESTAMP '2018-03-25 03:17:17.000'", "TIMESTAMP '2018-03-25 03:17:17.000'") - // time gap in Kathmandu - .addRoundTrip("TIMESTAMP '1986-01-01 00:13:07.000'", "TIMESTAMP '1986-01-01 00:13:07.000'") - - // same as above but with higher precision - .addRoundTrip("TIMESTAMP '1958-01-01 13:18:03.1230000'", "TIMESTAMP '1958-01-01 13:18:03.1230000'") - .addRoundTrip("TIMESTAMP '2019-03-18 10:01:17.9870000'", "TIMESTAMP '2019-03-18 10:01:17.9870000'") - .addRoundTrip("TIMESTAMP '2018-10-28 01:33:17.4560000'", "TIMESTAMP '2018-10-28 01:33:17.4560000'") - .addRoundTrip("TIMESTAMP '2018-10-28 03:33:33.3330000'", "TIMESTAMP '2018-10-28 03:33:33.3330000'") - .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.0000000'", "TIMESTAMP '1970-01-01 00:00:00.0000000'") - .addRoundTrip("TIMESTAMP '1970-01-01 00:13:42.0000000'", "TIMESTAMP '1970-01-01 00:13:42.0000000'") - .addRoundTrip("TIMESTAMP '2018-04-01 02:13:55.1230000'", "TIMESTAMP '2018-04-01 02:13:55.1230000'") - .addRoundTrip("TIMESTAMP '2018-03-25 03:17:17.0000000'", "TIMESTAMP '2018-03-25 03:17:17.0000000'") - .addRoundTrip("TIMESTAMP '1986-01-01 00:13:07.0000000'", "TIMESTAMP '1986-01-01 00:13:07.0000000'") - - // test arbitrary time for all supported precisions - .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00'", "TIMESTAMP '1970-01-01 00:00:00'") - .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.1'", "TIMESTAMP '1970-01-01 00:00:00.1'") - .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.12'", "TIMESTAMP '1970-01-01 00:00:00.12'") - .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.123'", "TIMESTAMP '1970-01-01 00:00:00.123'") - .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.1234'", "TIMESTAMP '1970-01-01 00:00:00.1234'") - .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.12345'", "TIMESTAMP '1970-01-01 00:00:00.12345'") - .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.123456'", "TIMESTAMP '1970-01-01 00:00:00.123456'") - .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.1234567'", "TIMESTAMP '1970-01-01 00:00:00.1234567'") - .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.12345670'", "TIMESTAMP '1970-01-01 00:00:00.1234567'") - .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.123456749999'", "TIMESTAMP '1970-01-01 00:00:00.1234567'") - .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.12345675'", "TIMESTAMP '1970-01-01 00:00:00.1234568'") - .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.12345679'", "TIMESTAMP '1970-01-01 00:00:00.1234568'") - - // before epoch with second fraction - .addRoundTrip("TIMESTAMP '1969-12-31 23:59:59.1230000'", "TIMESTAMP '1969-12-31 23:59:59.1230000'") - .addRoundTrip("TIMESTAMP '1969-12-31 23:59:59.1234567'", "TIMESTAMP '1969-12-31 23:59:59.1234567'") - - // precision 0 ends up as precision 0 - .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00'", "TIMESTAMP '1970-01-01 00:00:00'") - - .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.1'", "TIMESTAMP '1970-01-01 00:00:00.1'") - .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.9'", "TIMESTAMP '1970-01-01 00:00:00.9'") - .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.123'", "TIMESTAMP '1970-01-01 00:00:00.123'") - .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.123000'", "TIMESTAMP '1970-01-01 00:00:00.123000'") - .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.999'", "TIMESTAMP '1970-01-01 00:00:00.999'") - // max supported precision - .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.1234567'", "TIMESTAMP '1970-01-01 00:00:00.1234567'") - - .addRoundTrip("TIMESTAMP '2020-09-27 12:34:56.1'", "TIMESTAMP '2020-09-27 12:34:56.1'") - .addRoundTrip("TIMESTAMP '2020-09-27 12:34:56.9'", "TIMESTAMP '2020-09-27 12:34:56.9'") - .addRoundTrip("TIMESTAMP '2020-09-27 12:34:56.123'", "TIMESTAMP '2020-09-27 12:34:56.123'") - .addRoundTrip("TIMESTAMP '2020-09-27 12:34:56.123000'", "TIMESTAMP '2020-09-27 12:34:56.123000'") - .addRoundTrip("TIMESTAMP '2020-09-27 12:34:56.999'", "TIMESTAMP '2020-09-27 12:34:56.999'") - // max supported precision - .addRoundTrip("TIMESTAMP '2020-09-27 12:34:56.1234567'", "TIMESTAMP '2020-09-27 12:34:56.1234567'") - - // round down - .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.12345671'", "TIMESTAMP '1970-01-01 00:00:00.1234567'") - - // nanos round up, end result rounds down - .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.1234567499'", "TIMESTAMP '1970-01-01 00:00:00.1234567'") - .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.123456749999'", "TIMESTAMP '1970-01-01 00:00:00.1234567'") - - // round up - .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.12345675'", "TIMESTAMP '1970-01-01 00:00:00.1234568'") - - // max precision - .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.111222333444'", "TIMESTAMP '1970-01-01 00:00:00.1112223'") - - // round up to next second - .addRoundTrip("TIMESTAMP '1970-01-01 00:00:00.99999995'", "TIMESTAMP '1970-01-01 00:00:01.0000000'") - - // round up to next day - .addRoundTrip("TIMESTAMP '1970-01-01 23:59:59.99999995'", "TIMESTAMP '1970-01-02 00:00:00.0000000'") - - // negative epoch - .addRoundTrip("TIMESTAMP '1969-12-31 23:59:59.99999995'", "TIMESTAMP '1970-01-01 00:00:00.0000000'") - .addRoundTrip("TIMESTAMP '1969-12-31 23:59:59.999999949999'", "TIMESTAMP '1969-12-31 23:59:59.9999999'") - .addRoundTrip("TIMESTAMP '1969-12-31 23:59:59.99999994'", "TIMESTAMP '1969-12-31 23:59:59.9999999'"); - - Session session = Session.builder(getSession()) - .setTimeZoneKey(TimeZoneKey.getTimeZoneKey(sessionZone.getId())) - .build(); - - tests.execute(getQueryRunner(), session, trinoCreateAsSelect(session, "test_timestamp")); - tests.execute(getQueryRunner(), session, trinoCreateAsSelect(getSession(), "test_timestamp")); - tests.execute(getQueryRunner(), session, trinoCreateAndInsert(session, "test_timestamp")); - } - - @Test - public void testSqlServerDatetime2() - { - SqlDataTypeTest.create() - // literal values with higher precision are NOT rounded and cause an error - .addRoundTrip("DATETIME2(0)", "'1970-01-01 00:00:00'", createTimestampType(0), "TIMESTAMP '1970-01-01 00:00:00'") - .addRoundTrip("DATETIME2(1)", "'1970-01-01 00:00:00.1'", createTimestampType(1), "TIMESTAMP '1970-01-01 00:00:00.1'") - .addRoundTrip("DATETIME2(1)", "'1970-01-01 00:00:00.9'", createTimestampType(1), "TIMESTAMP '1970-01-01 00:00:00.9'") - .addRoundTrip("DATETIME2(3)", "'1970-01-01 00:00:00.123'", createTimestampType(3), "TIMESTAMP '1970-01-01 00:00:00.123'") - .addRoundTrip("DATETIME2(6)", "'1970-01-01 00:00:00.123000'", createTimestampType(6), "TIMESTAMP '1970-01-01 00:00:00.123000'") - .addRoundTrip("DATETIME2(3)", "'1970-01-01 00:00:00.999'", createTimestampType(3), "TIMESTAMP '1970-01-01 00:00:00.999'") - .addRoundTrip("DATETIME2(7)", "'1970-01-01 00:00:00.1234567'", createTimestampType(7), "TIMESTAMP '1970-01-01 00:00:00.1234567'") - .addRoundTrip("DATETIME2(1)", "'2020-09-27 12:34:56.1'", createTimestampType(1), "TIMESTAMP '2020-09-27 12:34:56.1'") - .addRoundTrip("DATETIME2(1)", "'2020-09-27 12:34:56.9'", createTimestampType(1), "TIMESTAMP '2020-09-27 12:34:56.9'") - .addRoundTrip("DATETIME2(3)", "'2020-09-27 12:34:56.123'", createTimestampType(3), "TIMESTAMP '2020-09-27 12:34:56.123'") - .addRoundTrip("DATETIME2(6)", "'2020-09-27 12:34:56.123000'", createTimestampType(6), "TIMESTAMP '2020-09-27 12:34:56.123000'") - .addRoundTrip("DATETIME2(3)", "'2020-09-27 12:34:56.999'", createTimestampType(3), "TIMESTAMP '2020-09-27 12:34:56.999'") - .addRoundTrip("DATETIME2(7)", "'2020-09-27 12:34:56.1234567'", createTimestampType(7), "TIMESTAMP '2020-09-27 12:34:56.1234567'") - - .addRoundTrip("DATETIME2(7)", "'1970-01-01 00:00:00'", createTimestampType(7), "TIMESTAMP '1970-01-01 00:00:00.0000000'") - .addRoundTrip("DATETIME2(7)", "'1970-01-01 00:00:00.1'", createTimestampType(7), "TIMESTAMP '1970-01-01 00:00:00.1000000'") - .addRoundTrip("DATETIME2(7)", "'1970-01-01 00:00:00.9'", createTimestampType(7), "TIMESTAMP '1970-01-01 00:00:00.9000000'") - .addRoundTrip("DATETIME2(7)", "'1970-01-01 00:00:00.123'", createTimestampType(7), "TIMESTAMP '1970-01-01 00:00:00.1230000'") - .addRoundTrip("DATETIME2(7)", "'1970-01-01 00:00:00.123000'", createTimestampType(7), "TIMESTAMP '1970-01-01 00:00:00.1230000'") - .addRoundTrip("DATETIME2(7)", "'1970-01-01 00:00:00.999'", createTimestampType(7), "TIMESTAMP '1970-01-01 00:00:00.9990000'") - .addRoundTrip("DATETIME2(7)", "'1970-01-01 00:00:00.1234567'", createTimestampType(7), "TIMESTAMP '1970-01-01 00:00:00.1234567'") - .addRoundTrip("DATETIME2(7)", "'2020-09-27 12:34:56.1'", createTimestampType(7), "TIMESTAMP '2020-09-27 12:34:56.1000000'") - .addRoundTrip("DATETIME2(7)", "'2020-09-27 12:34:56.9'", createTimestampType(7), "TIMESTAMP '2020-09-27 12:34:56.9000000'") - .addRoundTrip("DATETIME2(7)", "'2020-09-27 12:34:56.123'", createTimestampType(7), "TIMESTAMP '2020-09-27 12:34:56.1230000'") - .addRoundTrip("DATETIME2(7)", "'2020-09-27 12:34:56.123000'", createTimestampType(7), "TIMESTAMP '2020-09-27 12:34:56.1230000'") - .addRoundTrip("DATETIME2(7)", "'2020-09-27 12:34:56.999'", createTimestampType(7), "TIMESTAMP '2020-09-27 12:34:56.9990000'") - .addRoundTrip("DATETIME2(7)", "'2020-09-27 12:34:56.1234567'", createTimestampType(7), "TIMESTAMP '2020-09-27 12:34:56.1234567'") - - .execute(getQueryRunner(), sqlServerCreateAndInsert("test_sqlserver_timestamp")); - } - - @DataProvider - public Object[][] testTimestampDataProvider() - { - return new Object[][] { - {UTC}, - {ZoneId.systemDefault()}, - // using two non-JVM zones so that we don't need to worry what SQL Server system zone is - // no DST in 1970, but has DST in later years (e.g. 2018) - {ZoneId.of("Europe/Vilnius")}, - // minutes offset change since 1970-01-01, no DST - {ZoneId.of("Asia/Kathmandu")}, - {ZoneId.of(TestingSession.DEFAULT_TIME_ZONE_KEY.getId())}, - }; - } - - private DataSetup trinoCreateAsSelect(String tableNamePrefix) - { - return trinoCreateAsSelect(getSession(), tableNamePrefix); - } - - private DataSetup trinoCreateAsSelect(Session session, String tableNamePrefix) - { - return new CreateAsSelectDataSetup(new TrinoSqlExecutor(getQueryRunner(), session), tableNamePrefix); - } - - private DataSetup trinoCreateAndInsert(Session session, String tableNamePrefix) - { - return new CreateAndInsertDataSetup(new TrinoSqlExecutor(getQueryRunner(), session), tableNamePrefix); - } - - private DataSetup sqlServerCreateAndInsert(String tableNamePrefix) + @Override + protected SqlExecutor onRemoteDatabase() { - Properties properties = new Properties(); - properties.setProperty("user", sqlServer.getUsername()); - properties.setProperty("password", sqlServer.getPassword()); - return new CreateAndInsertDataSetup(new JdbcSqlExecutor(sqlServer.getJdbcUrl(), properties), tableNamePrefix); + return sqlServer::execute; } } diff --git a/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/TestSqlServerWithSnapshotIsolation.java b/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/TestSqlServerWithSnapshotIsolation.java new file mode 100644 index 000000000000..a69f127ac836 --- /dev/null +++ b/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/TestSqlServerWithSnapshotIsolation.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 io.trino.plugin.sqlserver; + +import static java.lang.String.format; + +public class TestSqlServerWithSnapshotIsolation + extends BaseSqlServerTransactionIsolationTest +{ + @Override + protected void configureDatabase(TestingSqlServer sqlServer) + { + String databaseName = sqlServer.getDatabaseName(); + + // ALLOW_SNAPSHOT_ISOLATION controls whether SNAPSHOT ISOLATION is actually enabled + sqlServer.execute(format("ALTER DATABASE %s SET ALLOW_SNAPSHOT_ISOLATION ON", databaseName)); + + // READ_COMMITTED_SNAPSHOT that READ COMMITTED transaction isolation uses SNAPSHOT ISOLATION by default + // it has no effect when ALLOW_SNAPSHOT_ISOLATION is disabled + // https://docs.microsoft.com/en-us/dotnet/framework/data/adonet/sql/snapshot-isolation-in-sql-server#snapshot-isolation-level-extensions + sqlServer.execute(format("ALTER DATABASE %s SET READ_COMMITTED_SNAPSHOT OFF", databaseName)); + } +} diff --git a/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/TestSqlServerWithoutSnapshotIsolation.java b/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/TestSqlServerWithoutSnapshotIsolation.java index c71961eeada5..8da557614a8f 100644 --- a/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/TestSqlServerWithoutSnapshotIsolation.java +++ b/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/TestSqlServerWithoutSnapshotIsolation.java @@ -13,73 +13,22 @@ */ package io.trino.plugin.sqlserver; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import io.trino.testing.AbstractTestQueryFramework; -import io.trino.testing.MaterializedResult; -import io.trino.testing.QueryRunner; -import io.trino.tpch.TpchTable; -import org.testng.annotations.Test; - -import static io.trino.plugin.sqlserver.SqlServerQueryRunner.createSqlServerQueryRunner; -import static io.trino.spi.type.VarcharType.VARCHAR; -import static org.assertj.core.api.Assertions.assertThat; +import static java.lang.String.format; public class TestSqlServerWithoutSnapshotIsolation - extends AbstractTestQueryFramework + extends BaseSqlServerTransactionIsolationTest { @Override - protected QueryRunner createQueryRunner() - throws Exception - { - TestingSqlServer sqlServer = closeAfterClass(new TestingSqlServer(false)); - sqlServer.start(); - return createSqlServerQueryRunner( - sqlServer, - ImmutableMap.of(), - ImmutableMap.of(), - ImmutableList.of(TpchTable.NATION)); - } - - @Test - public void testCreateReadTable() - { - assertUpdate("CREATE TABLE ctas_read AS SELECT * FROM tpch.tiny.nation", "SELECT count(*) FROM nation"); - assertQuery("SELECT AVG(LENGTH(name)) FROM ctas_read", "SELECT 7.08"); - assertQuery("SELECT SUM(LENGTH(name)) FROM ctas_read WHERE regionkey = 1", "SELECT 38"); - assertUpdate("DROP TABLE ctas_read"); - } - - @Test - public void testDescribeShowTable() + protected void configureDatabase(TestingSqlServer sqlServer) { - assertUpdate("CREATE TABLE ctas_describe AS SELECT regionkey, nationkey, comment FROM tpch.tiny.nation", "SELECT count(*) FROM nation"); + String databaseName = sqlServer.getDatabaseName(); - MaterializedResult expectedColumns = MaterializedResult.resultBuilder(getQueryRunner().getDefaultSession(), VARCHAR, VARCHAR, VARCHAR, VARCHAR) - .row("regionkey", "bigint", "", "") - .row("nationkey", "bigint", "", "") - .row("comment", "varchar(152)", "", "") - .build(); + // ALLOW_SNAPSHOT_ISOLATION controls whether SNAPSHOT ISOLATION is actually enabled + sqlServer.execute(format("ALTER DATABASE %s SET ALLOW_SNAPSHOT_ISOLATION OFF", databaseName)); - MaterializedResult actualColumns = computeActual("DESCRIBE ctas_describe"); - assertThat(actualColumns).isEqualTo(expectedColumns); - - MaterializedResult expectedTables = MaterializedResult.resultBuilder(getQueryRunner().getDefaultSession(), VARCHAR) - .row("ctas_describe") - .build(); - - MaterializedResult actualTables = computeActual("SHOW TABLES LIKE 'ctas_describe'"); - assertThat(actualTables).isEqualTo(expectedTables); - - assertUpdate("DROP TABLE ctas_describe"); - } - - @Test - public void testCreateInsertReadTable() - { - assertUpdate("CREATE TABLE insert_table (col INTEGER)"); - assertUpdate("INSERT INTO insert_table (col) VALUES (1), (2), (3), (4)", 4); - assertQuery("SELECT AVG(col) FROM insert_table", "SELECT 2.5"); - assertUpdate("DROP TABLE insert_table"); + // READ_COMMITTED_SNAPSHOT that READ COMMITTED transaction isolation uses SNAPSHOT ISOLATION by default + // it has no effect when ALLOW_SNAPSHOT_ISOLATION is disabled + // https://docs.microsoft.com/en-us/dotnet/framework/data/adonet/sql/snapshot-isolation-in-sql-server#snapshot-isolation-level-extensions + sqlServer.execute(format("ALTER DATABASE %s SET READ_COMMITTED_SNAPSHOT ON", databaseName)); } } diff --git a/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/TestSqlServerWithoutSnapshotIsolationAndReadCommittedSnapshot.java b/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/TestSqlServerWithoutSnapshotIsolationAndReadCommittedSnapshot.java new file mode 100644 index 000000000000..72b96ddd4ecc --- /dev/null +++ b/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/TestSqlServerWithoutSnapshotIsolationAndReadCommittedSnapshot.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 io.trino.plugin.sqlserver; + +import static java.lang.String.format; + +public class TestSqlServerWithoutSnapshotIsolationAndReadCommittedSnapshot + extends BaseSqlServerTransactionIsolationTest +{ + @Override + protected void configureDatabase(TestingSqlServer sqlServer) + { + String databaseName = sqlServer.getDatabaseName(); + + // ALLOW_SNAPSHOT_ISOLATION controls whether SNAPSHOT ISOLATION is actually enabled + sqlServer.execute(format("ALTER DATABASE %s SET ALLOW_SNAPSHOT_ISOLATION OFF", databaseName)); + + // READ_COMMITTED_SNAPSHOT that READ COMMITTED transaction isolation uses SNAPSHOT ISOLATION by default + // it has no effect when ALLOW_SNAPSHOT_ISOLATION is disabled + // https://docs.microsoft.com/en-us/dotnet/framework/data/adonet/sql/snapshot-isolation-in-sql-server#snapshot-isolation-level-extensions + sqlServer.execute(format("ALTER DATABASE %s SET READ_COMMITTED_SNAPSHOT OFF", databaseName)); + } +} diff --git a/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/TestingSqlServer.java b/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/TestingSqlServer.java index a4b6e2611c48..332835b2fd78 100644 --- a/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/TestingSqlServer.java +++ b/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/TestingSqlServer.java @@ -33,23 +33,21 @@ public final class TestingSqlServer private static final DockerImageName DOCKER_IMAGE_NAME = DockerImageName.parse("microsoft/mssql-server-linux:2017-CU13") .asCompatibleSubstituteFor("mcr.microsoft.com/mssql/server:2017-CU12"); private final MSSQLServerContainer container; - private final boolean snapshotIsolationEnabled; private final String databaseName; private Closeable cleanup = () -> {}; public TestingSqlServer() - { - this(true); - } - - public TestingSqlServer(boolean snapshotIsolationEnabled) { container = new MSSQLServerContainer<>(DOCKER_IMAGE_NAME); container.addEnv("ACCEPT_EULA", "yes"); - this.snapshotIsolationEnabled = snapshotIsolationEnabled; this.databaseName = "database_" + UUID.randomUUID().toString().replace("-", ""); } + public String getDatabaseName() + { + return databaseName; + } + public void execute(String sql) { try (Connection connection = container.createConnection(""); @@ -92,10 +90,9 @@ private void setUpDatabase() { execute("CREATE DATABASE " + databaseName); - if (snapshotIsolationEnabled) { - execute(format("ALTER DATABASE %s SET READ_COMMITTED_SNAPSHOT ON", databaseName)); - execute(format("ALTER DATABASE %s SET ALLOW_SNAPSHOT_ISOLATION ON", databaseName)); - } + // Enable snapshot isolation by default to reduce flakiness on CI + execute(format("ALTER DATABASE %s SET ALLOW_SNAPSHOT_ISOLATION ON", databaseName)); + execute(format("ALTER DATABASE %s SET READ_COMMITTED_SNAPSHOT ON", databaseName)); container.withUrlParam("database", this.databaseName); } diff --git a/plugin/trino-teradata-functions/pom.xml b/plugin/trino-teradata-functions/pom.xml index 5b5bab774922..a59e5a32d6bb 100644 --- a/plugin/trino-teradata-functions/pom.xml +++ b/plugin/trino-teradata-functions/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-thrift-api/pom.xml b/plugin/trino-thrift-api/pom.xml index 1484424908a7..a9a8e40a6c36 100644 --- a/plugin/trino-thrift-api/pom.xml +++ b/plugin/trino-thrift-api/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-thrift-testing-server/pom.xml b/plugin/trino-thrift-testing-server/pom.xml index 5b4fb5225d4f..31ae52d277b8 100644 --- a/plugin/trino-thrift-testing-server/pom.xml +++ b/plugin/trino-thrift-testing-server/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-thrift/pom.xml b/plugin/trino-thrift/pom.xml index f91420d2218b..1809ac40e2ba 100644 --- a/plugin/trino-thrift/pom.xml +++ b/plugin/trino-thrift/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-tpcds/pom.xml b/plugin/trino-tpcds/pom.xml index 6a03b9e41e49..c3b98501b02b 100644 --- a/plugin/trino-tpcds/pom.xml +++ b/plugin/trino-tpcds/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-tpch/pom.xml b/plugin/trino-tpch/pom.xml index 855965b1c1ac..5c24cca6083f 100644 --- a/plugin/trino-tpch/pom.xml +++ b/plugin/trino-tpch/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/pom.xml b/pom.xml index 0c4b6a2b9e69..bab987d9ff9c 100644 --- a/pom.xml +++ b/pom.xml @@ -5,12 +5,12 @@ io.airlift airbase - 107 + 110 io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT trino-root Trino @@ -47,7 +47,7 @@ 1.7.4 2.7.7-1 4.9 - 203 + 204 ${dep.airlift.version} 1.11.946 3.14.9 @@ -61,7 +61,7 @@ 2.5.1 1.15.1 3.2.7 - 1.0.33 + 1.0.42 5.5.2 + + net.bytebuddy + byte-buddy + 1.10.22 + + net.java.dev.jna diff --git a/service/trino-proxy/pom.xml b/service/trino-proxy/pom.xml index 7f5ae27cc5af..09367392d540 100644 --- a/service/trino-proxy/pom.xml +++ b/service/trino-proxy/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/service/trino-verifier/pom.xml b/service/trino-verifier/pom.xml index 9bc4ad3e8e50..faa62b471de6 100644 --- a/service/trino-verifier/pom.xml +++ b/service/trino-verifier/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/testing/trino-benchmark-driver/pom.xml b/testing/trino-benchmark-driver/pom.xml index f77a050f3934..e4a37a9251a0 100644 --- a/testing/trino-benchmark-driver/pom.xml +++ b/testing/trino-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/testing/trino-benchmark/pom.xml b/testing/trino-benchmark/pom.xml index 468505af73cc..8b6d71316057 100644 --- a/testing/trino-benchmark/pom.xml +++ b/testing/trino-benchmark/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/testing/trino-benchmark/src/main/java/io/trino/benchmark/HashBuildAndJoinBenchmark.java b/testing/trino-benchmark/src/main/java/io/trino/benchmark/HashBuildAndJoinBenchmark.java index ea5d79420932..0eecf87c5989 100644 --- a/testing/trino-benchmark/src/main/java/io/trino/benchmark/HashBuildAndJoinBenchmark.java +++ b/testing/trino-benchmark/src/main/java/io/trino/benchmark/HashBuildAndJoinBenchmark.java @@ -137,6 +137,7 @@ protected List createDrivers(TaskContext taskContext) lookupSourceFactoryManager, sourceTypes, false, + false, Ints.asList(0), hashChannel, Optional.empty(), diff --git a/testing/trino-benchmark/src/main/java/io/trino/benchmark/HashBuildBenchmark.java b/testing/trino-benchmark/src/main/java/io/trino/benchmark/HashBuildBenchmark.java index b9786ab8be4a..bb2eb8d59a86 100644 --- a/testing/trino-benchmark/src/main/java/io/trino/benchmark/HashBuildBenchmark.java +++ b/testing/trino-benchmark/src/main/java/io/trino/benchmark/HashBuildBenchmark.java @@ -96,6 +96,7 @@ protected List createDrivers(TaskContext taskContext) lookupSourceFactoryManager, ImmutableList.of(BIGINT), false, + false, Ints.asList(0), OptionalInt.empty(), Optional.empty(), diff --git a/testing/trino-benchmark/src/main/java/io/trino/benchmark/HashJoinBenchmark.java b/testing/trino-benchmark/src/main/java/io/trino/benchmark/HashJoinBenchmark.java index 6b4211aa80fa..6de480bf58b3 100644 --- a/testing/trino-benchmark/src/main/java/io/trino/benchmark/HashJoinBenchmark.java +++ b/testing/trino-benchmark/src/main/java/io/trino/benchmark/HashJoinBenchmark.java @@ -106,6 +106,7 @@ protected List createDrivers(TaskContext taskContext) lookupSourceFactoryManager, lineItemTypes, false, + false, Ints.asList(0), OptionalInt.empty(), Optional.empty(), diff --git a/testing/trino-benchto-benchmarks/pom.xml b/testing/trino-benchto-benchmarks/pom.xml index 78468bedc363..8eb5945f09ef 100644 --- a/testing/trino-benchto-benchmarks/pom.xml +++ b/testing/trino-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/testing/trino-product-tests-launcher/pom.xml b/testing/trino-product-tests-launcher/pom.xml index 1045d2d49c60..3d526270c0db 100644 --- a/testing/trino-product-tests-launcher/pom.xml +++ b/testing/trino-product-tests-launcher/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/DockerContainer.java b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/DockerContainer.java index 4c1007b5b0ab..25bd0d226203 100644 --- a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/DockerContainer.java +++ b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/DockerContainer.java @@ -238,6 +238,16 @@ private void copyFileToContainer(String containerPath, CheckedRunnable copy) } public String execCommand(String... command) + { + ExecResult result = execCommandForResult(command); + if (result.getExitCode() == 0) { + return result.getStdout(); + } + String fullCommand = Joiner.on(" ").join(command); + throw new RuntimeException(format("Could not execute command '%s' in container %s: %s", fullCommand, logicalName, result.getStderr())); + } + + public ExecResult execCommandForResult(String... command) { String fullCommand = Joiner.on(" ").join(command); if (!isRunning()) { @@ -247,12 +257,7 @@ public String execCommand(String... command) log.info("Executing command '%s' in container %s", fullCommand, logicalName); try { - ExecResult result = (ExecResult) executor.getAsync(() -> execInContainer(command)).get(); - if (result.getExitCode() == 0) { - return result.getStdout(); - } - - throw new RuntimeException(format("Could not execute command '%s' in container %s: %s", fullCommand, logicalName, result.getStderr())); + return (ExecResult) executor.getAsync(() -> execInContainer(command)).get(); } catch (InterruptedException e) { Thread.currentThread().interrupt(); @@ -348,9 +353,14 @@ private void copyFileFromContainer(String filename, Path targetPath) private List listFilesInContainer(String path) { try { + ExecResult execResult = execCommandForResult("/usr/bin/find", path, "-type", "f", "-print"); + if (execResult.getExitCode() != 0) { + log.warn("Could not list files in container '%s' path %s: %s", logicalName, path, execResult.getStderr()); + return ImmutableList.of(); + } return Splitter.on("\n") .omitEmptyStrings() - .splitToList(execCommand("/usr/bin/find", path, "-type", "f", "-print")); + .splitToList(execResult.getStdout()); } catch (RuntimeException e) { log.warn(e, "Could not list files in container '%s' path %s", logicalName, path); diff --git a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/EnvironmentModule.java b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/EnvironmentModule.java index 51d750c4523d..b8c42a1c1a00 100644 --- a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/EnvironmentModule.java +++ b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/EnvironmentModule.java @@ -23,6 +23,7 @@ import io.trino.tests.product.launcher.env.common.HadoopKerberosKms; import io.trino.tests.product.launcher.env.common.HydraIdentityProvider; import io.trino.tests.product.launcher.env.common.Kafka; +import io.trino.tests.product.launcher.env.common.KafkaSsl; import io.trino.tests.product.launcher.env.common.SeleniumChrome; import io.trino.tests.product.launcher.env.common.Standard; import io.trino.tests.product.launcher.env.common.StandardMultinode; @@ -64,6 +65,7 @@ public void configure(Binder binder) binder.bind(HadoopKerberosKms.class).in(SINGLETON); binder.bind(HydraIdentityProvider.class).in(SINGLETON); binder.bind(Kafka.class).in(SINGLETON); + binder.bind(KafkaSsl.class).in(SINGLETON); binder.bind(SeleniumChrome.class).in(SINGLETON); binder.bind(Standard.class).in(SINGLETON); binder.bind(StandardMultinode.class).in(SINGLETON); diff --git a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/common/Kafka.java b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/common/Kafka.java index 55ff9b48888e..11b0ca722b3e 100644 --- a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/common/Kafka.java +++ b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/common/Kafka.java @@ -30,6 +30,9 @@ public class Kafka { private static final String CONFLUENT_VERSION = "5.5.2"; private static final int SCHEMA_REGISTRY_PORT = 8081; + static final String KAFKA = "kafka"; + static final String SCHEMA_REGISTRY = "schema-registry"; + static final String ZOOKEEPER = "zookeeper"; private final PortBinder portBinder; @@ -43,14 +46,14 @@ public Kafka(PortBinder portBinder) public void extendEnvironment(Environment.Builder builder) { builder.addContainers(createZookeeper(), createKafka(), createSchemaRegistry()) - .containerDependsOn("kafka", "zookeeper") - .containerDependsOn("schema-registry", "kafka"); + .containerDependsOn(KAFKA, ZOOKEEPER) + .containerDependsOn(SCHEMA_REGISTRY, KAFKA); } @SuppressWarnings("resource") private DockerContainer createZookeeper() { - DockerContainer container = new DockerContainer("confluentinc/cp-zookeeper:" + CONFLUENT_VERSION, "zookeeper") + DockerContainer container = new DockerContainer("confluentinc/cp-zookeeper:" + CONFLUENT_VERSION, ZOOKEEPER) .withEnv("ZOOKEEPER_CLIENT_PORT", "2181") .withEnv("ZOOKEEPER_TICK_TIME", "2000") .withStartupCheckStrategy(new IsRunningStartupCheckStrategy()) @@ -64,7 +67,7 @@ private DockerContainer createZookeeper() @SuppressWarnings("resource") private DockerContainer createKafka() { - DockerContainer container = new DockerContainer("confluentinc/cp-kafka:" + CONFLUENT_VERSION, "kafka") + DockerContainer container = new DockerContainer("confluentinc/cp-kafka:" + CONFLUENT_VERSION, KAFKA) .withEnv("KAFKA_BROKER_ID", "1") .withEnv("KAFKA_ZOOKEEPER_CONNECT", "zookeeper:2181") .withEnv("KAFKA_ADVERTISED_LISTENERS", "PLAINTEXT://kafka:9092") @@ -81,7 +84,7 @@ private DockerContainer createKafka() @SuppressWarnings("resource") private DockerContainer createSchemaRegistry() { - DockerContainer container = new DockerContainer("confluentinc/cp-schema-registry:" + CONFLUENT_VERSION, "schema-registry") + DockerContainer container = new DockerContainer("confluentinc/cp-schema-registry:" + CONFLUENT_VERSION, SCHEMA_REGISTRY) .withEnv("SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS", "PLAINTEXT://kafka:9092") .withEnv("SCHEMA_REGISTRY_HOST_NAME", "0.0.0.0") .withEnv("SCHEMA_REGISTRY_LISTENERS", "http://0.0.0.0:" + SCHEMA_REGISTRY_PORT) diff --git a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/common/KafkaSsl.java b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/common/KafkaSsl.java new file mode 100644 index 000000000000..f15f6bf9edf8 --- /dev/null +++ b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/common/KafkaSsl.java @@ -0,0 +1,77 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.tests.product.launcher.env.common; + +import com.google.common.collect.ImmutableList; +import io.trino.tests.product.launcher.env.Environment; +import org.testcontainers.containers.BindMode; + +import javax.inject.Inject; + +import java.util.List; + +import static java.util.Objects.requireNonNull; + +public class KafkaSsl + implements EnvironmentExtender +{ + private final Kafka kafka; + + @Inject + public KafkaSsl(Kafka kafka) + { + this.kafka = requireNonNull(kafka, "kafka is null"); + } + + @Override + public void extendEnvironment(Environment.Builder builder) + { + builder.configureContainer(Kafka.KAFKA, container -> container.withEnv("KAFKA_ADVERTISED_LISTENERS", "SSL://kafka:9092") + .withEnv("KAFKA_SSL_KEYSTORE_FILENAME", "kafka.broker1.keystore") + .withEnv("KAFKA_SSL_KEYSTORE_CREDENTIALS", "broker1_keystore_creds") + .withEnv("KAFKA_SSL_KEYSTORE_TYPE", "JKS") + .withEnv("KAFKA_SSL_KEY_CREDENTIALS", "broker1_sslkey_creds") + .withEnv("KAFKA_SSL_TRUSTSTORE_FILENAME", "kafka.broker1.truststore") + .withEnv("KAFKA_SSL_TRUSTSTORE_CREDENTIALS", "broker1_truststore_creds") + .withEnv("KAFKA_SSL_TRUSTSTORE_TYPE", "JKS") + .withEnv("KAFKA_SSL_ENDPOINT_IDENTIFICATION_ALGORITHM", "https") + .withEnv("KAFKA_SSL_CLIENT_AUTH", "required") + .withEnv("KAFKA_SECURITY_INTER_BROKER_PROTOCOL", "SSL") + .withClasspathResourceMapping("docker/presto-product-tests/conf/environment/multinode-kafka-ssl/secrets", "/etc/kafka/secrets", BindMode.READ_ONLY)); + builder.configureContainer(Kafka.SCHEMA_REGISTRY, container -> container.withEnv("SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS", "SSL://kafka:9092") + .withEnv("SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL", "SSL") + .withEnv("SCHEMA_REGISTRY_KAFKASTORE_SSL_KEYSTORE_LOCATION", "/var/private/ssl/kafka.client.keystore") + .withEnv("SCHEMA_REGISTRY_KAFKASTORE_SSL_KEYSTORE_PASSWORD", "confluent") + .withEnv("SCHEMA_REGISTRY_KAFKASTORE_SSL_KEYSTORE_TYPE", "JKS") + .withEnv("SCHEMA_REGISTRY_KAFKASTORE_SSL_TRUSTSTORE_LOCATION", "/var/private/ssl/kafka.client.truststore") + .withEnv("SCHEMA_REGISTRY_KAFKASTORE_SSL_TRUSTSTORE_PASSWORD", "confluent") + .withEnv("SCHEMA_REGISTRY_KAFKASTORE_SSL_TRUSTSTORE_TYPE", "JKS") + .withEnv("SCHEMA_REGISTRY_KAFKASTORE_SSL_KEY_PASSWORD", "confluent") + .withEnv("SCHEMA_REGISTRY_KAFKASTORE_SSL_ENDPOINT_IDENTIFICATION_ALGORITHM", "https") + .withEnv("SCHEMA_REGISTRY_KAFKASTORE_SSL_CLIENT_AUTH", "requested") + .withEnv("SCHEMA_REGISTRY_SSL_KEYSTORE_LOCATION", "/var/private/ssl/kafka.client.keystore") + .withEnv("SCHEMA_REGISTRY_SSL_KEYSTORE_PASSWORD", "confluent") + .withEnv("SCHEMA_REGISTRY_SSL_KEYSTORE_TYPE", "JKS") + .withEnv("SCHEMA_REGISTRY_SSL_KEY_PASSWORD", "confluent") + .withEnv("SCHEMA_REGISTRY_SSL_ENDPOINT_IDENTIFICATION_ALGORITHM", "https") + .withClasspathResourceMapping("docker/presto-product-tests/conf/environment/multinode-kafka-ssl/secrets", "/var/private/ssl", BindMode.READ_ONLY)); + } + + @Override + public List getDependencies() + { + return ImmutableList.of(kafka); + } +} diff --git a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/AbstractSinglenodeLdap.java b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/AbstractSinglenodeLdap.java index 63e43f40a160..faae632d708d 100644 --- a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/AbstractSinglenodeLdap.java +++ b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/AbstractSinglenodeLdap.java @@ -39,7 +39,7 @@ public abstract class AbstractSinglenodeLdap extends EnvironmentProvider { - private final DockerFiles dockerFiles; + protected final DockerFiles dockerFiles; private final PortBinder portBinder; private final String imagesVersion; diff --git a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/MultinodeKafkaSsl.java b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/MultinodeKafkaSsl.java new file mode 100644 index 000000000000..f681a73c90cf --- /dev/null +++ b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/MultinodeKafkaSsl.java @@ -0,0 +1,77 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.tests.product.launcher.env.environment; + +import io.trino.tests.product.launcher.docker.DockerFiles; +import io.trino.tests.product.launcher.docker.DockerFiles.ResourceProvider; +import io.trino.tests.product.launcher.env.DockerContainer; +import io.trino.tests.product.launcher.env.Environment; +import io.trino.tests.product.launcher.env.EnvironmentProvider; +import io.trino.tests.product.launcher.env.common.KafkaSsl; +import io.trino.tests.product.launcher.env.common.StandardMultinode; +import io.trino.tests.product.launcher.env.common.TestsEnvironment; + +import javax.inject.Inject; + +import static io.trino.tests.product.launcher.env.EnvironmentContainers.COORDINATOR; +import static io.trino.tests.product.launcher.env.EnvironmentContainers.TESTS; +import static io.trino.tests.product.launcher.env.EnvironmentContainers.WORKER; +import static io.trino.tests.product.launcher.env.common.Standard.CONTAINER_PRESTO_ETC; +import static java.util.Objects.requireNonNull; +import static org.testcontainers.utility.MountableFile.forHostPath; + +@TestsEnvironment +public final class MultinodeKafkaSsl + extends EnvironmentProvider +{ + private final ResourceProvider configDir; + + @Inject + public MultinodeKafkaSsl(KafkaSsl kafka, StandardMultinode standardMultinode, DockerFiles dockerFiles) + { + super(standardMultinode, kafka); + requireNonNull(dockerFiles, "dockerFiles is null"); + configDir = dockerFiles.getDockerFilesHostDirectory("conf/environment/multinode-kafka-ssl/"); + } + + @Override + public void extendEnvironment(Environment.Builder builder) + { + builder.configureContainer(COORDINATOR, this::addCatalogs); + builder.configureContainer(WORKER, this::addCatalogs); + + builder.configureContainer(TESTS, container -> container + .withCopyFileToContainer( + forHostPath(configDir.getPath("tempto-configuration.yaml")), + "/docker/presto-product-tests/conf/tempto/tempto-configuration-profile-config-file.yaml") + .withCopyFileToContainer( + forHostPath(configDir.getPath("secrets")), + "/docker/presto-product-tests/conf/tempto/secrets")); + } + + private void addCatalogs(DockerContainer container) + { + container + .withCopyFileToContainer( + forHostPath(configDir.getPath("kafka_schema_registry.properties")), + CONTAINER_PRESTO_ETC + "/catalog/kafka_schema_registry.properties") + .withCopyFileToContainer( + forHostPath(configDir.getPath("kafka.properties")), + CONTAINER_PRESTO_ETC + "/catalog/kafka.properties") + .withCopyFileToContainer( + forHostPath(configDir.getPath("secrets")), + CONTAINER_PRESTO_ETC + "/catalog/secrets"); + } +} diff --git a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/SinglenodeLdapAndFile.java b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/SinglenodeLdapAndFile.java new file mode 100644 index 000000000000..9c2b299af8d2 --- /dev/null +++ b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/SinglenodeLdapAndFile.java @@ -0,0 +1,80 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.tests.product.launcher.env.environment; + +import com.google.common.collect.ImmutableList; +import io.trino.tests.product.launcher.docker.DockerFiles; +import io.trino.tests.product.launcher.docker.DockerFiles.ResourceProvider; +import io.trino.tests.product.launcher.env.Environment; +import io.trino.tests.product.launcher.env.EnvironmentConfig; +import io.trino.tests.product.launcher.env.common.Hadoop; +import io.trino.tests.product.launcher.env.common.Standard; +import io.trino.tests.product.launcher.env.common.TestsEnvironment; +import io.trino.tests.product.launcher.testcontainers.PortBinder; + +import javax.inject.Inject; + +import static io.trino.tests.product.launcher.env.EnvironmentContainers.COORDINATOR; +import static io.trino.tests.product.launcher.env.EnvironmentContainers.TESTS; +import static io.trino.tests.product.launcher.env.common.Standard.CONTAINER_PRESTO_CONFIG_PROPERTIES; +import static io.trino.tests.product.launcher.env.common.Standard.CONTAINER_PRESTO_ETC; +import static org.testcontainers.utility.MountableFile.forHostPath; + +@TestsEnvironment +public class SinglenodeLdapAndFile + extends AbstractSinglenodeLdap +{ + @Inject + public SinglenodeLdapAndFile(Standard standard, Hadoop hadoop, DockerFiles dockerFiles, PortBinder portBinder, EnvironmentConfig config) + { + super(ImmutableList.of(standard, hadoop), dockerFiles, portBinder, config); + } + + @Override + public void extendEnvironment(Environment.Builder builder) + { + super.extendEnvironment(builder); + ResourceProvider configDir = dockerFiles.getDockerFilesHostDirectory("conf/environment/singlenode-ldap-and-file"); + builder.configureContainer(COORDINATOR, dockerContainer -> { + dockerContainer + .withCopyFileToContainer( + forHostPath(configDir.getPath("config.properties")), + CONTAINER_PRESTO_CONFIG_PROPERTIES) + .withCopyFileToContainer( + forHostPath(configDir.getPath("file-authenticator.properties")), + CONTAINER_PRESTO_ETC + "/file-authenticator.properties") + .withCopyFileToContainer( + forHostPath(configDir.getPath("password.db")), + CONTAINER_PRESTO_ETC + "/password.db"); + }); + + builder.configureContainer(TESTS, dockerContainer -> { + String temptoConfig = "/docker/presto-product-tests/conf/tempto/tempto-configuration-for-singlenode_ldap_and_file.yaml"; + dockerContainer + .withCopyFileToContainer( + forHostPath(configDir.getPath("tempto-configuration.yaml")), + temptoConfig) + .withEnv("TEMPTO_CONFIG_FILES", temptoConfigFiles -> + temptoConfigFiles + .map(files -> files + "," + temptoConfig) + .orElse(temptoConfig)); + }); + } + + @Override + protected String getPasswordAuthenticatorConfigPath() + { + return "conf/environment/singlenode-ldap/password-authenticator.properties"; + } +} diff --git a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/suite/suites/Suite6NonGeneric.java b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/suite/suites/Suite6NonGeneric.java index 7d0dce5992b5..91a2028b2fd4 100644 --- a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/suite/suites/Suite6NonGeneric.java +++ b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/suite/suites/Suite6NonGeneric.java @@ -17,10 +17,12 @@ import io.trino.tests.product.launcher.env.EnvironmentConfig; import io.trino.tests.product.launcher.env.EnvironmentDefaults; import io.trino.tests.product.launcher.env.environment.MultinodeKafka; +import io.trino.tests.product.launcher.env.environment.MultinodeKafkaSsl; import io.trino.tests.product.launcher.env.environment.SinglenodeCassandra; import io.trino.tests.product.launcher.env.environment.SinglenodeKerberosKmsHdfsImpersonation; import io.trino.tests.product.launcher.env.environment.SinglenodeKerberosKmsHdfsNoImpersonation; import io.trino.tests.product.launcher.env.environment.SinglenodeLdap; +import io.trino.tests.product.launcher.env.environment.SinglenodeLdapAndFile; import io.trino.tests.product.launcher.env.environment.SinglenodeLdapInsecure; import io.trino.tests.product.launcher.env.environment.SinglenodeLdapReferrals; import io.trino.tests.product.launcher.suite.Suite; @@ -41,11 +43,13 @@ public List getTestRuns(EnvironmentConfig config) return ImmutableList.of( testOnEnvironment(SinglenodeLdap.class).withGroups("ldap").build(), + testOnEnvironment(SinglenodeLdapAndFile.class).withGroups("ldap", "ldap_and_file", "ldap_cli", "ldap_and_file_cli").build(), testOnEnvironment(SinglenodeLdapInsecure.class).withGroups("ldap").build(), testOnEnvironment(SinglenodeLdapReferrals.class).withGroups("ldap").build(), testOnEnvironment(SinglenodeKerberosKmsHdfsNoImpersonation.class).withGroups("storage_formats").build(), testOnEnvironment(SinglenodeKerberosKmsHdfsImpersonation.class).withGroups("storage_formats").build(), testOnEnvironment(SinglenodeCassandra.class).withGroups("cassandra").build(), - testOnEnvironment(MultinodeKafka.class).withGroups("kafka").build()); + testOnEnvironment(MultinodeKafka.class).withGroups("kafka").build(), + testOnEnvironment(MultinodeKafkaSsl.class).withGroups("kafka").build()); } } diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/kafka.properties b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/kafka.properties new file mode 100644 index 000000000000..dd501fe1c792 --- /dev/null +++ b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/kafka.properties @@ -0,0 +1,28 @@ +connector.name=kafka +kafka.table-names=product_tests.read_simple_key_and_value,\ + product_tests.read_all_datatypes_raw,\ + product_tests.read_all_datatypes_csv,\ + product_tests.read_all_datatypes_json,\ + product_tests.read_all_datatypes_avro,\ + product_tests.read_all_null_avro,\ + product_tests.read_structural_datatype_avro,\ + product_tests.write_simple_key_and_value,\ + product_tests.write_all_datatypes_raw,\ + product_tests.write_all_datatypes_csv,\ + product_tests.write_all_datatypes_json,\ + product_tests.write_all_datatypes_avro,\ + product_tests.write_structural_datatype_avro,\ + product_tests.pushdown_partition,\ + product_tests.pushdown_offset,\ + product_tests.pushdown_create_time +kafka.nodes=kafka:9092 +kafka.table-description-dir=/docker/presto-product-tests/conf/presto/etc/catalog/kafka +kafka.security-protocol=SSL +kafka.ssl.keystore.type=JKS +kafka.ssl.keystore.location=/docker/presto-product-tests/conf/presto/etc/catalog/secrets/kafka.client.keystore +kafka.ssl.keystore.password=confluent +kafka.ssl.truststore.type=JKS +kafka.ssl.truststore.location=/docker/presto-product-tests/conf/presto/etc/catalog/secrets/kafka.client.truststore +kafka.ssl.truststore.password=confluent +kafka.ssl.key.password=confluent +kafka.ssl.endpoint-identification-algorithm=https diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/kafka_schema_registry.properties b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/kafka_schema_registry.properties new file mode 100644 index 000000000000..5fddd5c41db5 --- /dev/null +++ b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/kafka_schema_registry.properties @@ -0,0 +1,14 @@ +connector.name=kafka +kafka.nodes=kafka:9092 +kafka.table-description-supplier=confluent +kafka.confluent-schema-registry-url=http://schema-registry:8081 +kafka.default-schema=product_tests +kafka.security-protocol=SSL +kafka.ssl.keystore.type=JKS +kafka.ssl.keystore.location=/docker/presto-product-tests/conf/presto/etc/catalog/secrets/kafka.client.keystore +kafka.ssl.keystore.password=confluent +kafka.ssl.truststore.type=JKS +kafka.ssl.truststore.location=/docker/presto-product-tests/conf/presto/etc/catalog/secrets/kafka.client.truststore +kafka.ssl.truststore.password=confluent +kafka.ssl.key.password=confluent +kafka.ssl.endpoint-identification-algorithm=https diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/secrets/.gitignore b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/secrets/.gitignore new file mode 100644 index 000000000000..cf132b981a59 --- /dev/null +++ b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/secrets/.gitignore @@ -0,0 +1,12 @@ +# automate certs generation in a future PR instead of committing them in git +# docker run --rm -v "$PWD/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/secrets":/secrets -w /secrets confluentinc/cp-kafka ./create-certs.sh +# *_creds +# *.jks +*.crt +*.key +*.pem +*.srl +*.req +*.csr +*.old +*.attr diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/secrets/broker1_keystore_creds b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/secrets/broker1_keystore_creds new file mode 100644 index 000000000000..2321227364ef --- /dev/null +++ b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/secrets/broker1_keystore_creds @@ -0,0 +1 @@ +confluent diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/secrets/broker1_sslkey_creds b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/secrets/broker1_sslkey_creds new file mode 100644 index 000000000000..2321227364ef --- /dev/null +++ b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/secrets/broker1_sslkey_creds @@ -0,0 +1 @@ +confluent diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/secrets/broker1_truststore_creds b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/secrets/broker1_truststore_creds new file mode 100644 index 000000000000..2321227364ef --- /dev/null +++ b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/secrets/broker1_truststore_creds @@ -0,0 +1 @@ +confluent diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/secrets/client_keystore_creds b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/secrets/client_keystore_creds new file mode 100644 index 000000000000..2321227364ef --- /dev/null +++ b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/secrets/client_keystore_creds @@ -0,0 +1 @@ +confluent diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/secrets/client_sslkey_creds b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/secrets/client_sslkey_creds new file mode 100644 index 000000000000..2321227364ef --- /dev/null +++ b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/secrets/client_sslkey_creds @@ -0,0 +1 @@ +confluent diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/secrets/client_truststore_creds b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/secrets/client_truststore_creds new file mode 100644 index 000000000000..2321227364ef --- /dev/null +++ b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/secrets/client_truststore_creds @@ -0,0 +1 @@ +confluent diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/secrets/create-certs.sh b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/secrets/create-certs.sh new file mode 100755 index 000000000000..2cc25281fec9 --- /dev/null +++ b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/secrets/create-certs.sh @@ -0,0 +1,50 @@ +#!/usr/bin/env bash + +set -xeuo pipefail + +# Generate CA key +openssl req \ + -new \ + -x509 \ + -keyout ca-1.key \ + -out ca-1.crt \ + -days 9999 \ + -subj '/CN=kafka/OU=TEST/O=TRINO/L=Montreal/S=Qc/C=CA' \ + -addext "subjectAltName = DNS:kafka" \ + -passin pass:confluent \ + -passout pass:confluent + +# Verify cert +# openssl x509 -in ca-1.crt -text -noout + +for i in broker1 client; do + echo $i + # Create keystores + keytool -genkey -noprompt \ + -alias $i \ + -dname "CN=kafka, OU=TEST, O=TRINO, L=Montreal, S=Qc, C=CA" \ + -keystore kafka.$i.keystore \ + -keyalg RSA \ + -storepass confluent \ + -keypass confluent \ + -ext SAN=dns:kafka + + # Verify keystore + # keytool -list -v -keystore kafka.broker1.keystore -storepass confluent + + # Create CSR, sign the key and import back into keystore + keytool -keystore kafka.$i.keystore -alias $i -certreq -file $i.csr -storepass confluent -keypass confluent -ext SAN=dns:kafka -noprompt + + openssl x509 -req -CA ca-1.crt -CAkey ca-1.key -in $i.csr -out $i-ca1-signed.crt -days 9999 -CAcreateserial -passin pass:confluent + + keytool -keystore kafka.$i.keystore -alias CARoot -import -file ca-1.crt -storepass confluent -keypass confluent -ext SAN=dns:kafka -noprompt + + keytool -keystore kafka.$i.keystore -alias $i -import -file $i-ca1-signed.crt -storepass confluent -keypass confluent -ext SAN=dns:kafka -noprompt + + # Create truststore and import the CA cert. + keytool -keystore kafka.$i.truststore -alias CARoot -import -file ca-1.crt -storepass confluent -keypass confluent -ext SAN=dns:kafka -noprompt + + echo "confluent" >${i}_sslkey_creds + echo "confluent" >${i}_keystore_creds + echo "confluent" >${i}_truststore_creds +done diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/secrets/kafka.broker1.keystore b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/secrets/kafka.broker1.keystore new file mode 100644 index 000000000000..9c687ae4a69c Binary files /dev/null and b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/secrets/kafka.broker1.keystore differ diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/secrets/kafka.broker1.truststore b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/secrets/kafka.broker1.truststore new file mode 100644 index 000000000000..7c5ce513185a Binary files /dev/null and b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/secrets/kafka.broker1.truststore differ diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/secrets/kafka.client.keystore b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/secrets/kafka.client.keystore new file mode 100644 index 000000000000..ded1b8471d7a Binary files /dev/null and b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/secrets/kafka.client.keystore differ diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/secrets/kafka.client.truststore b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/secrets/kafka.client.truststore new file mode 100644 index 000000000000..5c9ee35ce94c Binary files /dev/null and b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/secrets/kafka.client.truststore differ diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/tempto-configuration.yaml b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/tempto-configuration.yaml new file mode 100644 index 000000000000..a67984bdc989 --- /dev/null +++ b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-kafka-ssl/tempto-configuration.yaml @@ -0,0 +1,17 @@ +schema-registry: + url: http://schema-registry:8081 + +databases: + kafka: + broker: + host: kafka + port: 9092 + security.protocol: "SSL" + ssl.endpoint.identification.algorithm: "https" + ssl.key.password: "confluent" + ssl.keystore.location: "/docker/presto-product-tests/conf/tempto/secrets/kafka.client.keystore" + ssl.keystore.password: "confluent" + ssl.keystore.type: "JKS" + ssl.truststore.location: "/docker/presto-product-tests/conf/tempto/secrets/kafka.client.truststore" + ssl.truststore.password: "confluent" + ssl.truststore.type: "JKS" diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/singlenode-ldap-and-file/config.properties b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/singlenode-ldap-and-file/config.properties new file mode 100644 index 000000000000..0c4907a7c754 --- /dev/null +++ b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/singlenode-ldap-and-file/config.properties @@ -0,0 +1,34 @@ +# +# WARNING +# ^^^^^^^ +# This configuration file is for development only and should NOT be used +# in production. For example configuration, see the Trino documentation. +# + +node.id=will-be-overwritten +node.environment=test +node.internal-address-source=FQDN + +coordinator=true +node-scheduler.include-coordinator=true +query.max-memory=1GB +query.max-memory-per-node=512MB +query.max-total-memory-per-node=1GB +discovery-server.enabled=true +discovery.uri=https://presto-master:8443 + +# LDAP specific properties +# https will have to be enabled for ldap authentication +http-server.http.enabled=false +http-server.https.port=8443 +http-server.https.enabled=true +http-server.https.keystore.path=/etc/openldap/certs/coordinator.jks +http-server.https.keystore.key=testldap +http-server.authentication.type=PASSWORD,CERTIFICATE + +password-authenticator.config-files=etc/password-authenticator.properties,etc/file-authenticator.properties + +internal-communication.https.required=true +internal-communication.shared-secret=internal-shared-secret +internal-communication.https.keystore.path=/etc/openldap/certs/coordinator.jks +internal-communication.https.keystore.key=testldap diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/singlenode-ldap-and-file/file-authenticator.properties b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/singlenode-ldap-and-file/file-authenticator.properties new file mode 100644 index 000000000000..affed7fe356a --- /dev/null +++ b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/singlenode-ldap-and-file/file-authenticator.properties @@ -0,0 +1,2 @@ +password-authenticator.name=file +file.password-file=etc/password.db diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/singlenode-ldap-and-file/password.db b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/singlenode-ldap-and-file/password.db new file mode 100644 index 000000000000..8c595a606a0e --- /dev/null +++ b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/singlenode-ldap-and-file/password.db @@ -0,0 +1,2 @@ +DefaultGroupUser:$2y$10$xA36wzOAnGWJmukr/zItyOrOyXPD6prgszOCN93MyFUpMAbGKcklm +OnlyFileUser:$2y$10$002FmBq75Pos5V5esmi2k.86UpK6BbJRYHt9fpehdj1pJ6xZy5c1S diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/singlenode-ldap-and-file/tempto-configuration.yaml b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/singlenode-ldap-and-file/tempto-configuration.yaml new file mode 100644 index 000000000000..328243c9c38c --- /dev/null +++ b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/singlenode-ldap-and-file/tempto-configuration.yaml @@ -0,0 +1,5 @@ +databases: + presto: + file_user_password: FILEPass123 + OnlyFileUser@presto: + file_user_password: secondFILEPass123 diff --git a/testing/trino-product-tests/pom.xml b/testing/trino-product-tests/pom.xml index 5968f97e8a0f..5d73528c3a27 100644 --- a/testing/trino-product-tests/pom.xml +++ b/testing/trino-product-tests/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/TestGroups.java b/testing/trino-product-tests/src/main/java/io/trino/tests/TestGroups.java index 5df062928dd2..eef485f1064c 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/TestGroups.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/TestGroups.java @@ -50,7 +50,9 @@ public final class TestGroups public static final String CASSANDRA = "cassandra"; public static final String SQL_SERVER = "sqlserver"; public static final String LDAP = "ldap"; + public static final String LDAP_AND_FILE = "ldap_and_file"; public static final String LDAP_CLI = "ldap_cli"; + public static final String LDAP_AND_FILE_CLI = "ldap_and_file_cli"; public static final String SKIP_ON_CDH = "skip_on_cdh"; public static final String HDP3_ONLY = "hdp3_only"; public static final String TLS = "tls"; diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/cli/TestTrinoLdapCli.java b/testing/trino-product-tests/src/main/java/io/trino/tests/cli/TestTrinoLdapCli.java index 90d78af01c17..f54267516d9a 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/cli/TestTrinoLdapCli.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/cli/TestTrinoLdapCli.java @@ -44,6 +44,7 @@ import static io.trino.tests.ImmutableLdapObjectDefinitions.SPECIAL_USER; import static io.trino.tests.ImmutableLdapObjectDefinitions.USER_IN_MULTIPLE_GROUPS; import static io.trino.tests.TestGroups.LDAP; +import static io.trino.tests.TestGroups.LDAP_AND_FILE_CLI; import static io.trino.tests.TestGroups.LDAP_CLI; import static io.trino.tests.TestGroups.PROFILE_SPECIFIC_TESTS; import static java.lang.String.format; @@ -55,6 +56,8 @@ public class TestTrinoLdapCli extends TrinoCliLauncher implements RequirementsProvider { + private static final String SELECT_FROM_NATION = "SELECT * FROM hive.default.nation;"; + @Inject(optional = true) @Named("databases.presto.cli_ldap_truststore_path") private String ldapTruststorePath; @@ -75,6 +78,14 @@ public class TestTrinoLdapCli @Named("databases.presto.cli_ldap_user_password") private String ldapUserPassword; + @Inject(optional = true) + @Named("databases.presto.file_user_password") + private String fileUserPassword; + + @Inject(optional = true) + @Named("databases.OnlyFileUser@presto.file_user_password") + private String onlyFileUserPassword; + public TestTrinoLdapCli() throws IOException {} @@ -104,7 +115,7 @@ public void shouldRunQueryWithLdap() { launchTrinoCliWithServerArgument(); trino.waitForPrompt(); - trino.getProcessInput().println("select * from hive.default.nation;"); + trino.getProcessInput().println(SELECT_FROM_NATION); assertThat(trimLines(trino.readLinesUntilPrompt())).containsAll(nationTableInteractiveLines); } @@ -112,7 +123,7 @@ public void shouldRunQueryWithLdap() public void shouldRunBatchQueryWithLdap() throws IOException { - launchTrinoCliWithServerArgument("--execute", "select * from hive.default.nation;"); + launchTrinoCliWithServerArgument("--execute", SELECT_FROM_NATION); assertThat(trimLines(trino.readRemainingOutputLines())).containsAll(nationTableBatchLines); } @@ -122,7 +133,7 @@ public void shouldRunQueryFromFileWithLdap() { File temporayFile = File.createTempFile("test-sql", null); temporayFile.deleteOnExit(); - Files.write("select * from hive.default.nation;\n", temporayFile, UTF_8); + Files.write(SELECT_FROM_NATION + "\n", temporayFile, UTF_8); launchTrinoCliWithServerArgument("--file", temporayFile.getAbsolutePath()); assertThat(trimLines(trino.readRemainingOutputLines())).containsAll(nationTableBatchLines); @@ -134,7 +145,7 @@ public void shouldPassQueryForLdapUserInMultipleGroups() { ldapUserName = USER_IN_MULTIPLE_GROUPS.getAttributes().get("cn"); - launchTrinoCliWithServerArgument("--catalog", "hive", "--schema", "default", "--execute", "select * from nation;"); + launchTrinoCliWithServerArgument("--catalog", "hive", "--schema", "default", "--execute", SELECT_FROM_NATION); assertThat(trimLines(trino.readRemainingOutputLines())).containsAll(nationTableBatchLines); } @@ -143,7 +154,7 @@ public void shouldFailQueryForLdapUserInChildGroup() throws IOException { ldapUserName = CHILD_GROUP_USER.getAttributes().get("cn"); - launchTrinoCliWithServerArgument("--catalog", "hive", "--schema", "default", "--execute", "select * from nation;"); + launchTrinoCliWithServerArgument("--catalog", "hive", "--schema", "default", "--execute", SELECT_FROM_NATION); assertThat(trimLines(trino.readRemainingErrorLines())).anySatisfy(line -> assertThat(line).contains(format("User [%s] not a member of an authorized group", ldapUserName))); } @@ -153,7 +164,7 @@ public void shouldFailQueryForLdapUserInParentGroup() throws IOException { ldapUserName = PARENT_GROUP_USER.getAttributes().get("cn"); - launchTrinoCliWithServerArgument("--catalog", "hive", "--schema", "default", "--execute", "select * from nation;"); + launchTrinoCliWithServerArgument("--catalog", "hive", "--schema", "default", "--execute", SELECT_FROM_NATION); assertThat(trimLines(trino.readRemainingErrorLines())).anySatisfy(line -> assertThat(line).contains(format("User [%s] not a member of an authorized group", ldapUserName))); } @@ -163,7 +174,7 @@ public void shouldFailQueryForOrphanLdapUser() throws IOException { ldapUserName = ORPHAN_USER.getAttributes().get("cn"); - launchTrinoCliWithServerArgument("--catalog", "hive", "--schema", "default", "--execute", "select * from nation;"); + launchTrinoCliWithServerArgument("--catalog", "hive", "--schema", "default", "--execute", SELECT_FROM_NATION); assertThat(trimLines(trino.readRemainingErrorLines())).anySatisfy(line -> assertThat(line).contains(format("User [%s] not a member of an authorized group", ldapUserName))); } @@ -173,7 +184,7 @@ public void shouldFailQueryForWrongLdapPassword() throws IOException { ldapUserPassword = "wrong_password"; - launchTrinoCliWithServerArgument("--execute", "select * from hive.default.nation;"); + launchTrinoCliWithServerArgument("--execute", SELECT_FROM_NATION); assertThat(trimLines(trino.readRemainingErrorLines())).anySatisfy(line -> assertThat(line).contains("Invalid credentials")); } @@ -183,7 +194,7 @@ public void shouldFailQueryForWrongLdapUser() throws IOException { ldapUserName = "invalid_user"; - launchTrinoCliWithServerArgument("--execute", "select * from hive.default.nation;"); + launchTrinoCliWithServerArgument("--execute", SELECT_FROM_NATION); assertThat(trimLines(trino.readRemainingErrorLines())).anySatisfy(line -> assertThat(line).contains("Access Denied")); } @@ -193,7 +204,7 @@ public void shouldFailQueryForEmptyUser() throws IOException { ldapUserName = ""; - launchTrinoCliWithServerArgument("--execute", "select * from hive.default.nation;"); + launchTrinoCliWithServerArgument("--execute", SELECT_FROM_NATION); assertThat(trimLines(trino.readRemainingErrorLines())).anySatisfy(line -> assertThat(line).contains("Malformed credentials: user is empty")); } @@ -206,7 +217,7 @@ public void shouldFailQueryForLdapWithoutPassword() "--truststore-path", ldapTruststorePath, "--truststore-password", ldapTruststorePassword, "--user", ldapUserName, - "--execute", "select * from hive.default.nation;"); + "--execute", SELECT_FROM_NATION); assertThat(trimLines(trino.readRemainingErrorLines())).anySatisfy(line -> assertThat(line).contains("Authentication failed: Unauthorized")); } @@ -216,7 +227,7 @@ public void shouldFailQueryForLdapWithoutHttps() throws IOException { ldapServerAddress = format("http://%s:8443", serverHost); - launchTrinoCliWithServerArgument("--execute", "select * from hive.default.nation;"); + launchTrinoCliWithServerArgument("--execute", SELECT_FROM_NATION); assertThat(trimLines(trino.readRemainingErrorLines())).anySatisfy(line -> assertThat(line).contains("Authentication using username/password requires HTTPS to be enabled")); skipAfterTestWithContext(); @@ -227,7 +238,7 @@ public void shouldFailForIncorrectTrustStore() throws IOException { ldapTruststorePassword = "wrong_password"; - launchTrinoCliWithServerArgument("--execute", "select * from hive.default.nation;"); + launchTrinoCliWithServerArgument("--execute", SELECT_FROM_NATION); assertThat(trimLines(trino.readRemainingErrorLines())).anySatisfy(line -> assertThat(line).contains("Keystore was tampered with, or password was incorrect")); skipAfterTestWithContext(); @@ -245,7 +256,7 @@ public void shouldPassForCredentialsWithSpecialCharacters() { ldapUserName = SPECIAL_USER.getAttributes().get("cn"); ldapUserPassword = SPECIAL_USER.getAttributes().get("userPassword"); - launchTrinoCliWithServerArgument("--catalog", "hive", "--schema", "default", "--execute", "select * from nation;"); + launchTrinoCliWithServerArgument("--catalog", "hive", "--schema", "default", "--execute", SELECT_FROM_NATION); assertThat(trimLines(trino.readRemainingOutputLines())).containsAll(nationTableBatchLines); } @@ -254,12 +265,35 @@ public void shouldFailForUserWithColon() throws IOException { ldapUserName = "UserWith:Colon"; - launchTrinoCliWithServerArgument("--execute", "select * from hive.default.nation;"); + launchTrinoCliWithServerArgument("--execute", SELECT_FROM_NATION); assertThat(trimLines(trino.readRemainingErrorLines())).anySatisfy(line -> assertThat(line).contains("Illegal character ':' found in username")); skipAfterTestWithContext(); } + @Test(groups = {LDAP_AND_FILE_CLI, PROFILE_SPECIFIC_TESTS}, timeOut = TIMEOUT) + public void shouldRunQueryWithFileAuthenticator() + throws IOException + { + ldapUserPassword = fileUserPassword; + launchTrinoCliWithServerArgument(); + trino.waitForPrompt(); + trino.getProcessInput().println(SELECT_FROM_NATION); + assertThat(trimLines(trino.readLinesUntilPrompt())).containsAll(nationTableInteractiveLines); + } + + @Test(groups = {LDAP_AND_FILE_CLI, PROFILE_SPECIFIC_TESTS}, timeOut = TIMEOUT) + public void shouldRunQueryForAnotherUserWithOnlyFileAuthenticator() + throws IOException + { + ldapUserName = "OnlyFileUser"; + ldapUserPassword = onlyFileUserPassword; + launchTrinoCliWithServerArgument(); + trino.waitForPrompt(); + trino.getProcessInput().println(SELECT_FROM_NATION); + assertThat(trimLines(trino.readLinesUntilPrompt())).containsAll(nationTableInteractiveLines); + } + private void launchTrinoCliWithServerArgument(String... arguments) throws IOException { diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/hive/AbstractTestHiveViews.java b/testing/trino-product-tests/src/main/java/io/trino/tests/hive/AbstractTestHiveViews.java index 1fb2a520d3ce..48325c99d7b2 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/hive/AbstractTestHiveViews.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/hive/AbstractTestHiveViews.java @@ -19,6 +19,7 @@ import io.trino.tempto.fulfillment.table.hive.tpch.ImmutableTpchTablesRequirements.ImmutableOrdersTable; import io.trino.tempto.query.QueryExecutor; import io.trino.tempto.query.QueryResult; +import io.trino.testng.services.Flaky; import io.trino.tests.utils.QueryExecutors; import org.testng.annotations.Test; @@ -168,6 +169,7 @@ public void testShowCreateView() * Test view containing IF, IN, LIKE, BETWEEN, CASE, COALESCE, operators, delimited and non-delimited columns, an inline comment */ @Test(groups = HIVE_VIEWS) + @Flaky(issue = "https://github.com/trinodb/trino/issues/7535", match = "FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask") public void testRichSqlSyntax() { onHive().executeQuery("DROP VIEW IF EXISTS view_with_rich_syntax"); @@ -297,7 +299,8 @@ public void testSelectFromHiveViewWithoutDefaultCatalogAndSchema() public void testTimestampHiveView() { onHive().executeQuery("DROP TABLE IF EXISTS timestamp_hive_table"); - onHive().executeQuery("CREATE TABLE timestamp_hive_table AS SELECT cast('1990-01-02 12:13:14.123456789' AS timestamp) ts"); + onHive().executeQuery("CREATE TABLE timestamp_hive_table (ts timestamp)"); + onHive().executeQuery("INSERT INTO timestamp_hive_table (ts) values ('1990-01-02 12:13:14.123456789')"); onHive().executeQuery("DROP VIEW IF EXISTS timestamp_hive_view"); onHive().executeQuery("CREATE VIEW timestamp_hive_view AS SELECT * FROM timestamp_hive_table"); diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/hive/TestHiveStorageFormats.java b/testing/trino-product-tests/src/main/java/io/trino/tests/hive/TestHiveStorageFormats.java index b8524372fcb1..d6f37b5e98da 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/hive/TestHiveStorageFormats.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/hive/TestHiveStorageFormats.java @@ -480,6 +480,42 @@ public void testSnappyCompressedParquetTableCreatedInHive() onHive().executeQuery("DROP TABLE " + tableName); } + @Test + public void testOrcStructsWithNonLowercaseFields() + throws SQLException + { + String tableName = "orc_structs_with_non_lowercase"; + + ensureDummyExists(); + onHive().executeQuery("DROP TABLE IF EXISTS " + tableName); + + onHive().executeQuery(format( + "CREATE TABLE %s (" + + " c_bigint BIGINT," + + " c_struct struct)" + + "STORED AS ORC ", + tableName)); + + onHive().executeQuery(format( + "INSERT INTO %s" + // insert with SELECT because hive does not support array/map/struct functions in VALUES + + " SELECT" + + " 1," + + " named_struct('testCustId', '1234', 'requestDate', 'some day')" + // some hive versions don't allow INSERT from SELECT without FROM + + " FROM dummy", + tableName)); + + setSessionProperty(onTrino().getConnection(), "hive.projection_pushdown_enabled", "true"); + assertThat(onTrino().executeQuery("SELECT c_struct.testCustId FROM " + tableName)).containsOnly(row("1234")); + assertThat(onTrino().executeQuery("SELECT c_struct.testcustid FROM " + tableName)).containsOnly(row("1234")); + assertThat(onTrino().executeQuery("SELECT c_struct.requestDate FROM " + tableName)).containsOnly(row("some day")); + setSessionProperty(onTrino().getConnection(), "hive.projection_pushdown_enabled", "false"); + assertThat(onTrino().executeQuery("SELECT c_struct.testCustId FROM " + tableName)).containsOnly(row("1234")); + assertThat(onTrino().executeQuery("SELECT c_struct.testcustid FROM " + tableName)).containsOnly(row("1234")); + assertThat(onTrino().executeQuery("SELECT c_struct.requestDate FROM " + tableName)).containsOnly(row("some day")); + } + @Test(dataProvider = "storageFormatsWithNanosecondPrecision") public void testTimestampCreatedFromHive(StorageFormat storageFormat) { diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/hive/TestHiveTransactionalTable.java b/testing/trino-product-tests/src/main/java/io/trino/tests/hive/TestHiveTransactionalTable.java index 626a8a9de0cb..dbbf94537b90 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/hive/TestHiveTransactionalTable.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/hive/TestHiveTransactionalTable.java @@ -66,6 +66,7 @@ import static io.trino.tests.utils.QueryExecutors.onTrino; import static java.lang.String.format; import static java.util.Locale.ENGLISH; +import static java.util.concurrent.TimeUnit.MINUTES; import static java.util.stream.Collectors.joining; import static java.util.stream.Collectors.toUnmodifiableList; @@ -160,7 +161,7 @@ private void doTestReadFullAcid(boolean isPartitioned, BucketingType bucketingTy assertThat(query("SELECT col, fcol FROM " + tableName + " WHERE col=20")).containsExactly(row(20, 3)); - compactTableAndWait(MINOR, tableName, hivePartitionString, Duration.valueOf("6m")); + compactTableAndWait(MINOR, tableName, hivePartitionString, new Duration(6, MINUTES)); assertThat(query(selectFromOnePartitionsSql)).containsExactly(row(20, 3), row(21, 1), row(22, 2)); // delete a row @@ -177,7 +178,7 @@ private void doTestReadFullAcid(boolean isPartitioned, BucketingType bucketingTy assertThat(query("SELECT col, fcol FROM " + tableName + " WHERE col=20")).containsExactly(row(20, 3)); // test major compaction - compactTableAndWait(MAJOR, tableName, hivePartitionString, Duration.valueOf("6m")); + compactTableAndWait(MAJOR, tableName, hivePartitionString, new Duration(6, MINUTES)); assertThat(query(selectFromOnePartitionsSql)).containsExactly(row(20, 3), row(23, 1)); } } @@ -212,7 +213,7 @@ public void testReadInsertOnly(boolean isPartitioned, BucketingType bucketingTyp assertThat(query("SELECT col FROM " + tableName + " WHERE col=2")).containsExactly(row(2)); // test minor compacted data read - compactTableAndWait(MINOR, tableName, hivePartitionString, Duration.valueOf("6m")); + compactTableAndWait(MINOR, tableName, hivePartitionString, new Duration(6, MINUTES)); assertThat(query(selectFromOnePartitionsSql)).containsExactly(row(1), row(2)); assertThat(query("SELECT col FROM " + tableName + " WHERE col=2")).containsExactly(row(2)); @@ -225,7 +226,7 @@ public void testReadInsertOnly(boolean isPartitioned, BucketingType bucketingTyp // test major compaction onHive().executeQuery("INSERT INTO TABLE " + tableName + hivePartitionString + " SELECT 4"); - compactTableAndWait(MAJOR, tableName, hivePartitionString, Duration.valueOf("6m")); + compactTableAndWait(MAJOR, tableName, hivePartitionString, new Duration(6, MINUTES)); assertThat(query(selectFromOnePartitionsSql)).containsOnly(row(3), row(4)); } } @@ -974,7 +975,56 @@ public void testAcidUpdateFailNonTransactional() log.info("About to fail update"); assertThat(() -> onTrino().executeQuery(format("UPDATE %s SET purchase = 'bread' WHERE customer = 'Fred'", tableName))) - .failsWithMessage("Hive update is only supported for transactional tables"); + .failsWithMessage("Hive update is only supported for ACID transactional tables"); + }); + } + + @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + public void testAcidUpdateFailInsertOnlyTable() + { + withTemporaryTable("update_fail_insert_only", true, false, NONE, tableName -> { + onHive().executeQuery("CREATE TABLE " + tableName + " (customer STRING, purchase STRING) " + + "STORED AS ORC " + + hiveTableProperties(INSERT_ONLY, NONE)); + + log.info("About to insert"); + onTrino().executeQuery(format("INSERT INTO %s (customer, purchase) VALUES ('Fred', 'cards')", tableName)); + + log.info("About to fail update"); + assertThat(() -> onTrino().executeQuery(format("UPDATE %s SET purchase = 'bread' WHERE customer = 'Fred'", tableName))) + .failsWithMessage("Hive update is only supported for ACID transactional tables"); + }); + } + + @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + public void testAcidDeleteFailNonTransactional() + { + withTemporaryTable("delete_fail_nontransactional", true, true, NONE, tableName -> { + onTrino().executeQuery(format("CREATE TABLE %s (customer VARCHAR, purchase VARCHAR)", tableName)); + + log.info("About to insert"); + onTrino().executeQuery(format("INSERT INTO %s (customer, purchase) VALUES ('Fred', 'cards')", tableName)); + + log.info("About to fail delete"); + assertThat(() -> onTrino().executeQuery(format("DELETE FROM %s WHERE customer = 'Fred'", tableName))) + .failsWithMessage("Deletes must match whole partitions for non-transactional tables"); + }); + } + + @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + public void testAcidDeleteFailInsertOnlyTable() + { + withTemporaryTable("delete_fail_insert_only", true, false, NONE, tableName -> { + onHive().executeQuery("CREATE TABLE " + tableName + " (customer STRING, purchase STRING) " + + "STORED AS ORC " + + hiveTableProperties(INSERT_ONLY, NONE)); + + log.info("About to insert"); + onTrino().executeQuery(format("INSERT INTO %s (customer, purchase) VALUES ('Fred', 'cards')", tableName)); + + log.info("About to fail delete"); + assertThat(() -> onTrino().executeQuery(format("DELETE FROM %s WHERE customer = 'Fred'", tableName))) + .failsWithMessage("Deletes must match whole partitions for non-transactional tables"); }); } @@ -1214,7 +1264,7 @@ public void testAcidUpdateMajorCompaction() onTrino().executeQuery(format("INSERT INTO %s VALUES (22, 200)", tableName)); verifySelectForPrestoAndHive("SELECT * FROM " + tableName, "true", row(11, 100L), row(22, 200L)); log.info("About to compact"); - compactTableAndWait(MAJOR, tableName, "", Duration.valueOf("6m")); + compactTableAndWait(MAJOR, tableName, "", new Duration(6, MINUTES)); log.info("About to update"); onTrino().executeQuery(format("UPDATE %s SET column1 = 33 WHERE column2 = 200", tableName)); log.info("About to select"); @@ -1473,7 +1523,7 @@ private static void compactTableAndWait(CompactionMode compactMode, String table throw new IllegalStateException(format("Could not compact table %s in %d retries", tableName, event.getAttemptCount()), event.getFailure()); }) .onSuccess(event -> log.info("Finished %s compaction on %s in %s (%d tries)", compactMode, tableName, event.getElapsedTime(), event.getAttemptCount())) - .run(() -> tryCompactingTable(compactMode, tableName, partitionString, Duration.valueOf("2m"))); + .run(() -> tryCompactingTable(compactMode, tableName, partitionString, new Duration(2, MINUTES))); } private static void tryCompactingTable(CompactionMode compactMode, String tableName, String partitionString, Duration timeout) diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/hive/TestHiveViews.java b/testing/trino-product-tests/src/main/java/io/trino/tests/hive/TestHiveViews.java index 89fadc6d8837..98b811e916dc 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/hive/TestHiveViews.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/hive/TestHiveViews.java @@ -118,4 +118,22 @@ public void testLateralViewExplode() row("nothing", null), row("zero", null))); } + + @Test(groups = HIVE_VIEWS) + public void testLateralViewJsonTupleAs() + { + onTrino().executeQuery("DROP TABLE IF EXISTS test_json_tuple_table"); + onTrino().executeQuery("" + + "CREATE TABLE test_json_tuple_table WITH (format='TEXTFILE') AS " + + "SELECT 3 id, CAST('{\"user_id\": 1000, \"first.name\": \"Mateusz\", \"Last Name\": \"Gajewski\", \".NET\": true, \"aNull\": null}' AS varchar) jsonstr"); + + onHive().executeQuery("DROP VIEW IF EXISTS test_json_tuple_view"); + onHive().executeQuery("CREATE VIEW test_json_tuple_view AS " + + "SELECT `t`.`id`, `x`.`a`, `x`.`b`, `x`.`c`, `x`.`d`, `x`.`e`, `x`.`f` FROM test_json_tuple_table AS `t` " + + "LATERAL VIEW json_tuple(`t`.`jsonstr`, \"first.name\", \"Last Name\", '.NET', \"user_id\", \"aNull\", \"nonexistentField\") `x` AS `a`, `b`, `c`, `d`, `e`, `f`"); + + assertViewQuery( + "SELECT * FROM test_json_tuple_view", + queryAssert -> queryAssert.containsOnly(row(3, "Mateusz", "Gajewski", "true", "1000", null, null))); + } } diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/jdbc/BaseLdapJdbcTest.java b/testing/trino-product-tests/src/main/java/io/trino/tests/jdbc/BaseLdapJdbcTest.java index bfc075861927..a01bcdddb8fb 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/jdbc/BaseLdapJdbcTest.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/jdbc/BaseLdapJdbcTest.java @@ -48,7 +48,7 @@ public abstract class BaseLdapJdbcTest { protected static final long TIMEOUT = 30 * 1000; // seconds per test - protected static final String NATION_SELECT_ALL_QUERY = "select * from tpch.tiny.nation"; + protected static final String NATION_SELECT_ALL_QUERY = "SELECT * FROM tpch.tiny.nation"; @Inject @Named("databases.presto.cli_ldap_truststore_path") diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/jdbc/TestLdapTrinoJdbc.java b/testing/trino-product-tests/src/main/java/io/trino/tests/jdbc/TestLdapTrinoJdbc.java index 474beadac757..198bc77ff042 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/jdbc/TestLdapTrinoJdbc.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/jdbc/TestLdapTrinoJdbc.java @@ -13,6 +13,8 @@ */ package io.trino.tests.jdbc; +import com.google.inject.Inject; +import com.google.inject.name.Named; import io.trino.tempto.Requires; import io.trino.tempto.fulfillment.table.hive.tpch.ImmutableTpchTablesRequirements.ImmutableNationTable; import org.testng.annotations.Test; @@ -25,6 +27,7 @@ import static io.trino.tests.ImmutableLdapObjectDefinitions.ORPHAN_USER; import static io.trino.tests.ImmutableLdapObjectDefinitions.PARENT_GROUP_USER; import static io.trino.tests.TestGroups.LDAP; +import static io.trino.tests.TestGroups.LDAP_AND_FILE; import static io.trino.tests.TestGroups.PROFILE_SPECIFIC_TESTS; import static io.trino.tests.TestGroups.TRINO_JDBC; import static io.trino.tests.TpchTableResults.PRESTO_NATION_RESULT; @@ -34,6 +37,14 @@ public class TestLdapTrinoJdbc extends BaseLdapJdbcTest { + @Inject(optional = true) + @Named("databases.presto.file_user_password") + private String fileUserPassword; + + @Inject(optional = true) + @Named("databases.OnlyFileUser@presto.file_user_password") + private String onlyFileUserPassword; + @Override protected String getLdapUrlFormat() { @@ -118,8 +129,33 @@ public void shouldFailForUserWithColon() expectQueryToFail("UserWith:Colon", ldapUserPassword, "Illegal character ':' found in username"); } + @Requires(ImmutableNationTable.class) + @Test(groups = {LDAP_AND_FILE, TRINO_JDBC, PROFILE_SPECIFIC_TESTS}, timeOut = TIMEOUT) + public void shouldRunQueryWithFileAuthenticator() + throws SQLException + { + assertThat(executeLdapQuery(NATION_SELECT_ALL_QUERY, ldapUserName, fileUserPassword)).matches(PRESTO_NATION_RESULT); + } + + @Requires(ImmutableNationTable.class) + @Test(groups = {LDAP_AND_FILE, TRINO_JDBC, PROFILE_SPECIFIC_TESTS}, timeOut = TIMEOUT) + public void shouldRunQueryForAnotherUserWithOnlyFileAuthenticator() + throws SQLException + { + assertThat(executeLdapQuery(NATION_SELECT_ALL_QUERY, "OnlyFileUser", onlyFileUserPassword)).matches(PRESTO_NATION_RESULT); + } + private void expectQueryToFailForUserNotInGroup(String user) { expectQueryToFail(user, ldapUserPassword, format("Authentication failed: Access Denied: User [%s] not a member of an authorized group", user)); } + + @Override + protected void expectQueryToFail(String user, String password, String message) + { + //Changes 'hasMessage' to 'hasMessageContaining' to pass multiple authenticators tests + assertThatThrownBy(() -> executeLdapQuery(NATION_SELECT_ALL_QUERY, user, password)) + .isInstanceOf(SQLException.class) + .hasMessageContaining(message); + } } diff --git a/testing/trino-product-tests/src/main/resources/sql-tests/testcases/horology_functions/checkHorologyFunctionsRegistered.result b/testing/trino-product-tests/src/main/resources/sql-tests/testcases/horology_functions/checkHorologyFunctionsRegistered.result index 535e8d3d1911..84eda77f0157 100644 --- a/testing/trino-product-tests/src/main/resources/sql-tests/testcases/horology_functions/checkHorologyFunctionsRegistered.result +++ b/testing/trino-product-tests/src/main/resources/sql-tests/testcases/horology_functions/checkHorologyFunctionsRegistered.result @@ -44,7 +44,7 @@ format_datetime | varchar | timestamp(p), varchar(x) | scalar | true | Formats the given time by the given format | from_iso8601_date | date | varchar(x) | scalar | true | | from_iso8601_timestamp | timestamp(3) with time zone | varchar(x) | scalar | true | | - from_unixtime | timestamp(3) | double | scalar | true | | + from_unixtime | timestamp(3) with time zone | double | scalar | true | | from_unixtime | timestamp(3) with time zone | double, bigint, bigint | scalar | true | | hour | bigint | interval day to second | scalar | true | Hour of the day of the given interval | hour | bigint | time(p) | scalar | true | Hour of the day of the given time | @@ -67,7 +67,6 @@ quarter | bigint | timestamp(p) with time zone | scalar | true | Quarter of the year of the given timestamp | timezone_hour | bigint | timestamp(p) with time zone | scalar | true | Time zone hour of the given timestamp | timezone_minute | bigint | timestamp(p) with time zone | scalar | true | Time zone minute of the given timestamp | - to_unixtime | double | timestamp(p) | scalar | true | | to_unixtime | double | timestamp(p) with time zone | scalar | true | | week | bigint | date | scalar | true | Week of the year of the given date | week | bigint | timestamp(p) | scalar | true | Week of the year of the given timestamp | diff --git a/testing/trino-server-dev/pom.xml b/testing/trino-server-dev/pom.xml index 418cb9ada601..22890b67d29a 100644 --- a/testing/trino-server-dev/pom.xml +++ b/testing/trino-server-dev/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/testing/trino-test-jdbc-compatibility-old-driver/pom.xml b/testing/trino-test-jdbc-compatibility-old-driver/pom.xml index a924a1d450c6..100eeab69fad 100644 --- a/testing/trino-test-jdbc-compatibility-old-driver/pom.xml +++ b/testing/trino-test-jdbc-compatibility-old-driver/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml @@ -14,7 +14,7 @@ ${project.parent.basedir} - 355-SNAPSHOT + 356-SNAPSHOT diff --git a/testing/trino-test-jdbc-compatibility-old-server/pom.xml b/testing/trino-test-jdbc-compatibility-old-server/pom.xml index 84068898577e..3fccb1a3447b 100644 --- a/testing/trino-test-jdbc-compatibility-old-server/pom.xml +++ b/testing/trino-test-jdbc-compatibility-old-server/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/testing/trino-testing-kafka/pom.xml b/testing/trino-testing-kafka/pom.xml index 660d4a7b5707..eba645ba9902 100644 --- a/testing/trino-testing-kafka/pom.xml +++ b/testing/trino-testing-kafka/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/testing/trino-testing/pom.xml b/testing/trino-testing/pom.xml index 18ebb0107c14..340536fb6f7c 100644 --- a/testing/trino-testing/pom.xml +++ b/testing/trino-testing/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestAggregations.java b/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestAggregations.java index fafad75b717d..8a0614ba207a 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestAggregations.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestAggregations.java @@ -736,24 +736,24 @@ public void testApproximateCountDistinct() assertQuery("SELECT approx_distinct(CAST(orderdate AS TIMESTAMP(9) WITH TIME ZONE), 0.023) FROM orders", "SELECT 2322"); // test time - assertQuery("SELECT approx_distinct(CAST(from_unixtime(custkey) AS TIME)) FROM orders", "SELECT 969"); - assertQuery("SELECT approx_distinct(CAST(from_unixtime(custkey) AS TIME), 0.023) FROM orders", "SELECT 969"); + assertQuery("SELECT approx_distinct(CAST(from_unixtime(custkey) AS TIME)) FROM orders", "SELECT 1005"); + assertQuery("SELECT approx_distinct(CAST(from_unixtime(custkey) AS TIME), 0.023) FROM orders", "SELECT 1005"); // test time(9) (long representation) - assertQuery("SELECT approx_distinct(CAST(from_unixtime(custkey) AS TIME(9))) FROM orders", "SELECT 969"); - assertQuery("SELECT approx_distinct(CAST(from_unixtime(custkey) AS TIME(9)), 0.023) FROM orders", "SELECT 969"); + assertQuery("SELECT approx_distinct(CAST(from_unixtime(custkey) AS TIME(9))) FROM orders", "SELECT 1005"); + assertQuery("SELECT approx_distinct(CAST(from_unixtime(custkey) AS TIME(9)), 0.023) FROM orders", "SELECT 1005"); Session session = Session.builder(getSession()) .setTimeZoneKey(TimeZoneKey.getTimeZoneKey("+08:35")) .build(); // test time with time zone - assertQuery(session, "SELECT approx_distinct(CAST(from_unixtime(custkey) AS TIME WITH TIME ZONE)) FROM orders", "SELECT 993"); - assertQuery(session, "SELECT approx_distinct(CAST(from_unixtime(custkey) AS TIME WITH TIME ZONE), 0.023) FROM orders", "SELECT 993"); + assertQuery(session, "SELECT approx_distinct(CAST(from_unixtime(custkey) AS TIME WITH TIME ZONE)) FROM orders", "SELECT 1013"); + assertQuery(session, "SELECT approx_distinct(CAST(from_unixtime(custkey) AS TIME WITH TIME ZONE), 0.023) FROM orders", "SELECT 1013"); // test time(12) with time zone (long representation) - assertQuery(session, "SELECT approx_distinct(CAST(from_unixtime(custkey) AS TIME(12) WITH TIME ZONE)) FROM orders", "SELECT 1000"); - assertQuery(session, "SELECT approx_distinct(CAST(from_unixtime(custkey) AS TIME(12) WITH TIME ZONE), 0.023) FROM orders", "SELECT 1000"); + assertQuery(session, "SELECT approx_distinct(CAST(from_unixtime(custkey) AS TIME(12) WITH TIME ZONE)) FROM orders", "SELECT 969"); + assertQuery(session, "SELECT approx_distinct(CAST(from_unixtime(custkey) AS TIME(12) WITH TIME ZONE), 0.023) FROM orders", "SELECT 969"); // test short decimal assertQuery("SELECT approx_distinct(CAST(custkey AS DECIMAL(18, 0))) FROM orders", "SELECT 990"); diff --git a/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestQueries.java b/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestQueries.java index e7921f4d8c61..dc85cd3be813 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestQueries.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestQueries.java @@ -165,8 +165,8 @@ public void testLimit() @Test public void testLimitWithAggregation() { - MaterializedResult actual = computeActual("SELECT custkey, SUM(CAST(totalprice * 100 AS BIGINT)) FROM orders GROUP BY custkey LIMIT 10"); - MaterializedResult all = computeExpected("SELECT custkey, SUM(CAST(totalprice * 100 AS BIGINT)) FROM orders GROUP BY custkey", actual.getTypes()); + MaterializedResult actual = computeActual("SELECT custkey, SUM(orderkey) FROM orders GROUP BY custkey LIMIT 10"); + MaterializedResult all = computeExpected("SELECT custkey, SUM(orderkey) FROM orders GROUP BY custkey", actual.getTypes()); assertEquals(actual.getMaterializedRows().size(), 10); assertContains(all, actual); diff --git a/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java b/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java index 5c9cd8e903e6..b9eb0b6077af 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java @@ -36,6 +36,7 @@ import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_INSERT; import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_RENAME_TABLE; import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_RENAME_TABLE_ACROSS_SCHEMAS; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_TOPN_PUSHDOWN; import static io.trino.testing.assertions.Assert.assertEquals; import static io.trino.testing.sql.TestTable.randomTableSuffix; import static java.lang.String.join; @@ -284,6 +285,19 @@ public void testPredicateReflectedInExplain() "(predicate|filterPredicate|constraint).{0,10}(nationkey|NATIONKEY)"); } + @Test + public void testSortItemsReflectedInExplain() + { + // Even if the sort items are pushed down into the table scan, it should still be reflected in EXPLAIN (via ConnectorTableHandle.toString) + @Language("RegExp") String expectedPattern = hasBehavior(SUPPORTS_TOPN_PUSHDOWN) + ? "sortOrder=\\[(?i:nationkey):.* DESC NULLS LAST] limit=5" + : "\\[5 by \\((?i:nationkey) DESC NULLS LAST\\)]"; + + assertExplain( + "EXPLAIN SELECT name FROM nation ORDER BY nationkey DESC NULLS LAST LIMIT 5", + expectedPattern); + } + @Test public void testConcurrentScans() { diff --git a/testing/trino-testing/src/main/java/io/trino/testing/CountingMockConnector.java b/testing/trino-testing/src/main/java/io/trino/testing/CountingMockConnector.java index 35522a7d7582..b71b07cdb24b 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/CountingMockConnector.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/CountingMockConnector.java @@ -34,6 +34,7 @@ import static com.google.common.collect.ImmutableList.toImmutableList; import static com.google.common.collect.ImmutableSet.toImmutableSet; import static io.trino.connector.MockConnectorFactory.Builder.defaultGetColumns; +import static io.trino.connector.MockConnectorFactory.Builder.defaultGetTableHandle; import static io.trino.spi.security.PrincipalType.USER; public class CountingMockConnector @@ -54,6 +55,7 @@ public class CountingMockConnector private final AtomicLong listSchemasCallsCounter = new AtomicLong(); private final AtomicLong listTablesCallsCounter = new AtomicLong(); + private final AtomicLong getTableHandleCallsCounter = new AtomicLong(); private final AtomicLong getColumnsCallsCounter = new AtomicLong(); private final ListRoleGrantsCounter listRoleGranstCounter = new ListRoleGrantsCounter(); @@ -81,13 +83,16 @@ public MetadataCallsCount runCounting(Runnable runnable) synchronized (lock) { listSchemasCallsCounter.set(0); listTablesCallsCounter.set(0); + getTableHandleCallsCounter.set(0); getColumnsCallsCounter.set(0); listRoleGranstCounter.reset(); runnable.run(); - return new MetadataCallsCount(listSchemasCallsCounter.get(), + return new MetadataCallsCount( + listSchemasCallsCounter.get(), listTablesCallsCounter.get(), + getTableHandleCallsCounter.get(), getColumnsCallsCounter.get(), listRoleGranstCounter.listRowGrantsCallsCounter.get(), listRoleGranstCounter.rolesPushedCounter.get(), @@ -113,6 +118,10 @@ private ConnectorFactory getConnectorFactory() } return ImmutableList.of(); }) + .withGetTableHandle((connectorSession, schemaTableName) -> { + getTableHandleCallsCounter.incrementAndGet(); + return defaultGetTableHandle().apply(connectorSession, schemaTableName); + }) .withGetColumns(schemaTableName -> { getColumnsCallsCounter.incrementAndGet(); return defaultGetColumns().apply(schemaTableName); @@ -130,6 +139,7 @@ public static final class MetadataCallsCount { private final long listSchemasCount; private final long listTablesCount; + private final long getTableHandleCount; private final long getColumnsCount; private final long listRoleGrantsCount; private final long rolesPushedCount; @@ -138,11 +148,13 @@ public static final class MetadataCallsCount public MetadataCallsCount() { - this(0, 0, 0, 0, 0, 0, 0); + this(0, 0, 0, 0, 0, 0, 0, 0); } - public MetadataCallsCount(long listSchemasCount, + public MetadataCallsCount( + long listSchemasCount, long listTablesCount, + long getTableHandleCount, long getColumnsCount, long listRoleGrantsCount, long rolesPushedCount, @@ -151,6 +163,7 @@ public MetadataCallsCount(long listSchemasCount, { this.listSchemasCount = listSchemasCount; this.listTablesCount = listTablesCount; + this.getTableHandleCount = getTableHandleCount; this.getColumnsCount = getColumnsCount; this.listRoleGrantsCount = listRoleGrantsCount; this.rolesPushedCount = rolesPushedCount; @@ -160,37 +173,106 @@ public MetadataCallsCount(long listSchemasCount, public MetadataCallsCount withListSchemasCount(long listSchemasCount) { - return new MetadataCallsCount(listSchemasCount, listTablesCount, getColumnsCount, listRoleGrantsCount, rolesPushedCount, granteesPushedCount, limitPushedCount); + return new MetadataCallsCount( + listSchemasCount, + listTablesCount, + getTableHandleCount, + getColumnsCount, + listRoleGrantsCount, + rolesPushedCount, + granteesPushedCount, + limitPushedCount); } public MetadataCallsCount withListTablesCount(long listTablesCount) { - return new MetadataCallsCount(listSchemasCount, listTablesCount, getColumnsCount, listRoleGrantsCount, rolesPushedCount, granteesPushedCount, limitPushedCount); + return new MetadataCallsCount( + listSchemasCount, + listTablesCount, + getTableHandleCount, + getColumnsCount, + listRoleGrantsCount, + rolesPushedCount, + granteesPushedCount, + limitPushedCount); + } + + public MetadataCallsCount withGetTableHandleCount(long getTableHandleCount) + { + return new MetadataCallsCount( + listSchemasCount, + listTablesCount, + getTableHandleCount, + getColumnsCount, + listRoleGrantsCount, + rolesPushedCount, + granteesPushedCount, + limitPushedCount); } public MetadataCallsCount withGetColumnsCount(long getColumnsCount) { - return new MetadataCallsCount(listSchemasCount, listTablesCount, getColumnsCount, listRoleGrantsCount, rolesPushedCount, granteesPushedCount, limitPushedCount); + return new MetadataCallsCount( + listSchemasCount, + listTablesCount, + getTableHandleCount, + getColumnsCount, + listRoleGrantsCount, + rolesPushedCount, + granteesPushedCount, + limitPushedCount); } public MetadataCallsCount withListRoleGrantsCount(long listRoleGrantsCount) { - return new MetadataCallsCount(listSchemasCount, listTablesCount, getColumnsCount, listRoleGrantsCount, rolesPushedCount, granteesPushedCount, limitPushedCount); + return new MetadataCallsCount( + listSchemasCount, + listTablesCount, + getTableHandleCount, + getColumnsCount, + listRoleGrantsCount, + rolesPushedCount, + granteesPushedCount, + limitPushedCount); } public MetadataCallsCount withRolesPushedCount(long rolesPushedCount) { - return new MetadataCallsCount(listSchemasCount, listTablesCount, getColumnsCount, listRoleGrantsCount, rolesPushedCount, granteesPushedCount, limitPushedCount); + return new MetadataCallsCount( + listSchemasCount, + listTablesCount, + getTableHandleCount, + getColumnsCount, + listRoleGrantsCount, + rolesPushedCount, + granteesPushedCount, + limitPushedCount); } public MetadataCallsCount withGranteesPushedCount(long granteesPushedCount) { - return new MetadataCallsCount(listSchemasCount, listTablesCount, getColumnsCount, listRoleGrantsCount, rolesPushedCount, granteesPushedCount, limitPushedCount); + return new MetadataCallsCount( + listSchemasCount, + listTablesCount, + getTableHandleCount, + getColumnsCount, + listRoleGrantsCount, + rolesPushedCount, + granteesPushedCount, + limitPushedCount); } public MetadataCallsCount withLimitPushedCount(long limitPushedCount) { - return new MetadataCallsCount(listSchemasCount, listTablesCount, getColumnsCount, listRoleGrantsCount, rolesPushedCount, granteesPushedCount, limitPushedCount); + return new MetadataCallsCount( + listSchemasCount, + listTablesCount, + getTableHandleCount, + getColumnsCount, + listRoleGrantsCount, + rolesPushedCount, + granteesPushedCount, + limitPushedCount); } @Override @@ -205,6 +287,7 @@ public boolean equals(Object o) MetadataCallsCount that = (MetadataCallsCount) o; return listSchemasCount == that.listSchemasCount && listTablesCount == that.listTablesCount && + getTableHandleCount == that.getTableHandleCount && getColumnsCount == that.getColumnsCount && listRoleGrantsCount == that.listRoleGrantsCount && rolesPushedCount == that.rolesPushedCount && @@ -215,8 +298,10 @@ public boolean equals(Object o) @Override public int hashCode() { - return Objects.hash(listSchemasCount, + return Objects.hash( + listSchemasCount, listTablesCount, + getTableHandleCount, getColumnsCount, listRoleGrantsCount, rolesPushedCount, @@ -230,6 +315,7 @@ public String toString() return toStringHelper(this) .add("listSchemasCount", listSchemasCount) .add("listTablesCount", listTablesCount) + .add("getTableHandleCount", getTableHandleCount) .add("getColumnsCount", getColumnsCount) .add("listRoleGrantsCount", listRoleGrantsCount) .add("rolesPushedCount", rolesPushedCount) diff --git a/testing/trino-testing/src/main/java/io/trino/testing/DistributedQueryRunner.java b/testing/trino-testing/src/main/java/io/trino/testing/DistributedQueryRunner.java index b3e3b09c21ba..26a30dccc856 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/DistributedQueryRunner.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/DistributedQueryRunner.java @@ -36,6 +36,7 @@ import io.trino.server.testing.TestingTrinoServer; import io.trino.spi.Plugin; import io.trino.spi.QueryId; +import io.trino.spi.eventlistener.EventListener; import io.trino.spi.security.SystemAccessControl; import io.trino.split.PageSourceManager; import io.trino.split.SplitManager; @@ -99,7 +100,8 @@ private DistributedQueryRunner( String environment, Module additionalModule, Optional baseDataDir, - List systemAccessControls) + List systemAccessControls, + List eventListeners) throws Exception { requireNonNull(defaultSession, "defaultSession is null"); @@ -124,6 +126,7 @@ private DistributedQueryRunner( environment, additionalModule, baseDataDir, + ImmutableList.of(), ImmutableList.of())); servers.add(worker); } @@ -146,7 +149,8 @@ private DistributedQueryRunner( environment, additionalModule, baseDataDir, - systemAccessControls)); + systemAccessControls, + eventListeners)); servers.add(coordinator); if (backupCoordinatorProperties.isPresent()) { Map extraBackupCoordinatorProperties = new HashMap<>(); @@ -159,7 +163,8 @@ private DistributedQueryRunner( environment, additionalModule, baseDataDir, - systemAccessControls))); + systemAccessControls, + eventListeners))); servers.add(backupCoordinator.get()); } else { @@ -197,7 +202,8 @@ private static TestingTrinoServer createTestingTrinoServer( String environment, Module additionalModule, Optional baseDataDir, - List systemAccessControls) + List systemAccessControls, + List eventListeners) { long start = System.nanoTime(); ImmutableMap.Builder propertiesBuilder = ImmutableMap.builder() @@ -232,6 +238,7 @@ private static TestingTrinoServer createTestingTrinoServer( .setAdditionalModule(additionalModule) .setBaseDataDir(baseDataDir) .setSystemAccessControls(systemAccessControls) + .setEventListeners(eventListeners) .build(); String nodeRole = coordinator ? "coordinator" : "worker"; @@ -253,6 +260,7 @@ public void addServers(int nodeCount) ENVIRONMENT, EMPTY_MODULE, Optional.empty(), + ImmutableList.of(), ImmutableList.of())); serverBuilder.add(server); // add functions @@ -564,6 +572,7 @@ public static class Builder private Module additionalModule = EMPTY_MODULE; private Optional baseDataDir = Optional.empty(); private List systemAccessControls = ImmutableList.of(); + private List eventListeners = ImmutableList.of(); protected Builder(Session defaultSession) { @@ -648,6 +657,19 @@ public Builder setSystemAccessControls(List systemAccessCon return this; } + @SuppressWarnings("unused") + public Builder setEventListener(EventListener eventListener) + { + return setEventListeners(ImmutableList.of(requireNonNull(eventListener, "eventListener is null"))); + } + + @SuppressWarnings("unused") + public Builder setEventListeners(List eventListeners) + { + this.eventListeners = ImmutableList.copyOf(requireNonNull(eventListeners, "eventListeners is null")); + return this; + } + public Builder enableBackupCoordinator() { if (backupCoordinatorProperties.isEmpty()) { @@ -668,7 +690,8 @@ public DistributedQueryRunner build() environment, additionalModule, baseDataDir, - systemAccessControls); + systemAccessControls, + eventListeners); } } } diff --git a/testing/trino-testing/src/main/java/io/trino/testing/TestingConnectorBehavior.java b/testing/trino-testing/src/main/java/io/trino/testing/TestingConnectorBehavior.java index 42ba56404e37..8683b4ae7df8 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/TestingConnectorBehavior.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/TestingConnectorBehavior.java @@ -26,6 +26,7 @@ public enum TestingConnectorBehavior SUPPORTS_LIMIT_PUSHDOWN, SUPPORTS_TOPN_PUSHDOWN, + SUPPORTS_TOPN_PUSHDOWN_WITH_VARCHAR(fallback -> fallback.test(SUPPORTS_TOPN_PUSHDOWN) && fallback.test(SUPPORTS_PREDICATE_PUSHDOWN_WITH_VARCHAR_INEQUALITY)), SUPPORTS_AGGREGATION_PUSHDOWN, @@ -56,6 +57,8 @@ public enum TestingConnectorBehavior SUPPORTS_ARRAY, + SUPPORTS_CANCELLATION(false), + /**/; private final Predicate> hasBehaviorByDefault; diff --git a/testing/trino-testing/src/main/java/io/trino/testing/sql/TestView.java b/testing/trino-testing/src/main/java/io/trino/testing/sql/TestView.java new file mode 100644 index 000000000000..9703df236b45 --- /dev/null +++ b/testing/trino-testing/src/main/java/io/trino/testing/sql/TestView.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 io.trino.testing.sql; + +import static io.trino.testing.sql.TestTable.randomTableSuffix; +import static java.lang.String.format; + +public class TestView + implements AutoCloseable +{ + private final SqlExecutor sqlExecutor; + private final String name; + + public TestView(SqlExecutor sqlExecutor, String viewBody) + { + this.sqlExecutor = sqlExecutor; + this.name = "test_view_" + randomTableSuffix(); + sqlExecutor.execute(format("CREATE VIEW %s AS %s", name, viewBody)); + } + + public String getName() + { + return name; + } + + @Override + public void close() + { + sqlExecutor.execute("DROP VIEW " + name); + } +} diff --git a/testing/trino-testing/src/test/java/io/trino/testing/TestTestingTrinoClient.java b/testing/trino-testing/src/test/java/io/trino/testing/TestTestingTrinoClient.java index ceab6973caa6..dba9183532cb 100644 --- a/testing/trino-testing/src/test/java/io/trino/testing/TestTestingTrinoClient.java +++ b/testing/trino-testing/src/test/java/io/trino/testing/TestTestingTrinoClient.java @@ -31,6 +31,9 @@ import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; import java.security.Principal; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -51,16 +54,20 @@ public class TestTestingTrinoClient @BeforeClass public void setup() + throws IOException { + Path passwordConfigDummy = Files.createTempFile("passwordConfigDummy", ""); + passwordConfigDummy.toFile().deleteOnExit(); server = TestingTrinoServer.builder() .setProperties(ImmutableMap.builder() + .put("password-authenticator.config-files", passwordConfigDummy.toString()) .put("http-server.authentication.type", "password") .put("http-server.authentication.allow-insecure-over-http", "false") .put("http-server.process-forwarded", "true") .build()) .build(); - server.getInstance(Key.get(PasswordAuthenticatorManager.class)).setAuthenticator(TestTestingTrinoClient::authenticate); + server.getInstance(Key.get(PasswordAuthenticatorManager.class)).setAuthenticators(TestTestingTrinoClient::authenticate); } private static Principal authenticate(String user, String password) diff --git a/testing/trino-testng-services/pom.xml b/testing/trino-testng-services/pom.xml index f0d14acc9609..8ae6f4494f29 100644 --- a/testing/trino-testng-services/pom.xml +++ b/testing/trino-testng-services/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/testing/trino-testng-services/src/main/java/io/trino/testng/services/LogTestDurationListener.java b/testing/trino-testng-services/src/main/java/io/trino/testng/services/LogTestDurationListener.java index 64aae87e14f3..e10821c497af 100644 --- a/testing/trino-testng-services/src/main/java/io/trino/testng/services/LogTestDurationListener.java +++ b/testing/trino-testng-services/src/main/java/io/trino/testng/services/LogTestDurationListener.java @@ -43,6 +43,8 @@ import static io.trino.testng.services.Listeners.reportListenerFailure; import static java.lang.String.format; import static java.lang.management.ManagementFactory.getThreadMXBean; +import static java.util.concurrent.TimeUnit.MINUTES; +import static java.util.concurrent.TimeUnit.SECONDS; import static java.util.stream.Collectors.joining; public class LogTestDurationListener @@ -50,10 +52,10 @@ public class LogTestDurationListener { private static final Logger LOG = Logger.get(LogTestDurationListener.class); - private static final Duration SINGLE_TEST_LOGGING_THRESHOLD = Duration.valueOf("30s"); - private static final Duration CLASS_LOGGING_THRESHOLD = Duration.valueOf("1m"); + private static final Duration SINGLE_TEST_LOGGING_THRESHOLD = new Duration(30, SECONDS); + private static final Duration CLASS_LOGGING_THRESHOLD = new Duration(1, MINUTES); // Must be below Travis "no output" timeout (10m). E.g. TestElasticsearchIntegrationSmokeTest is known to take ~5-6m. - private static final Duration GLOBAL_IDLE_LOGGING_THRESHOLD = Duration.valueOf("8m"); + private static final Duration GLOBAL_IDLE_LOGGING_THRESHOLD = new Duration(8, MINUTES); private final boolean enabled; private final ScheduledExecutorService scheduledExecutorService; diff --git a/testing/trino-tests/pom.xml b/testing/trino-tests/pom.xml index ee0668d6568b..ecfea56d3de6 100644 --- a/testing/trino-tests/pom.xml +++ b/testing/trino-tests/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 355-SNAPSHOT + 356-SNAPSHOT ../../pom.xml diff --git a/testing/trino-tests/src/test/java/io/trino/execution/TestEventListenerBasic.java b/testing/trino-tests/src/test/java/io/trino/execution/TestEventListenerBasic.java index 435a65be7a40..fb6dec2509fb 100644 --- a/testing/trino-tests/src/test/java/io/trino/execution/TestEventListenerBasic.java +++ b/testing/trino-tests/src/test/java/io/trino/execution/TestEventListenerBasic.java @@ -27,9 +27,13 @@ import io.trino.spi.QueryId; import io.trino.spi.connector.ColumnMetadata; import io.trino.spi.connector.ConnectorFactory; +import io.trino.spi.connector.ConnectorMaterializedViewDefinition; +import io.trino.spi.connector.ConnectorMaterializedViewDefinition.Column; import io.trino.spi.connector.ConnectorViewDefinition; import io.trino.spi.connector.SchemaTableName; +import io.trino.spi.eventlistener.ColumnDetail; import io.trino.spi.eventlistener.ColumnInfo; +import io.trino.spi.eventlistener.OutputColumnMetadata; import io.trino.spi.eventlistener.QueryCompletedEvent; import io.trino.spi.eventlistener.QueryCreatedEvent; import io.trino.spi.eventlistener.QueryFailureInfo; @@ -96,12 +100,20 @@ public Iterable getConnectorFactories() { MockConnectorFactory connectorFactory = MockConnectorFactory.builder() .withListTables((session, s) -> ImmutableList.of(new SchemaTableName("default", "tests_table"))) - .withGetColumns(schemaTableName -> ImmutableList.of(new ColumnMetadata("test_varchar", createUnboundedVarcharType()))) + .withGetColumns(schemaTableName -> ImmutableList.of( + new ColumnMetadata("test_varchar", createUnboundedVarcharType()), + new ColumnMetadata("test_bigint", BIGINT))) + .withGetTableHandle((session, schemaTableName) -> { + if (!schemaTableName.getTableName().startsWith("create")) { + return new MockConnectorTableHandle(schemaTableName); + } + return null; + }) .withApplyProjection((session, handle, projections, assignments) -> { - if (((MockConnectorTableHandle) handle).getTableName().getTableName().equals("test_table_with_column_mask")) { - return Optional.empty(); + if (((MockConnectorTableHandle) handle).getTableName().getTableName().equals("tests_table")) { + throw new RuntimeException("Throw from apply projection"); } - throw new RuntimeException("Throw from apply projection"); + return Optional.empty(); }) .withGetViews((connectorSession, prefix) -> { ConnectorViewDefinition definition = new ConnectorViewDefinition( @@ -115,6 +127,19 @@ public Iterable getConnectorFactories() SchemaTableName viewName = new SchemaTableName("default", "test_view"); return ImmutableMap.of(viewName, definition); }) + .withGetMaterializedViews((connectorSession, prefix) -> { + ConnectorMaterializedViewDefinition definition = new ConnectorMaterializedViewDefinition( + "SELECT nationkey AS test_column FROM tpch.tiny.nation", + Optional.empty(), + Optional.empty(), + Optional.empty(), + ImmutableList.of(new Column("test_column", BIGINT.getTypeId())), + Optional.empty(), + "alice", + ImmutableMap.of()); + SchemaTableName materializedViewName = new SchemaTableName("default", "test_materialized_view"); + return ImmutableMap.of(materializedViewName, definition); + }) .withRowFilter(schemaTableName -> { if (schemaTableName.getTableName().equals("test_table_with_row_filter")) { return new ViewExpression("user", Optional.of("tpch"), Optional.of("tiny"), "EXISTS (SELECT 1 FROM nation WHERE name = test_varchar)"); @@ -122,7 +147,7 @@ public Iterable getConnectorFactories() return null; }) .withColumnMask((schemaTableName, columnName) -> { - if (schemaTableName.getTableName().equals("test_table_with_column_mask")) { + if (schemaTableName.getTableName().equals("test_table_with_column_mask") && columnName.equals("test_varchar")) { return new ViewExpression("user", Optional.of("tpch"), Optional.of("tiny"), "(SELECT cast(max(orderkey) as VARCHAR) FROM orders)"); } return null; @@ -279,6 +304,7 @@ public void testReferencedTablesAndRoutines() TableInfo table = tables.get(0); assertEquals(table.getCatalog(), "tpch"); assertEquals(table.getSchema(), "tiny"); + assertEquals(table.getTable(), "lineitem"); assertEquals(table.getAuthorization(), "user"); assertTrue(table.getFilters().isEmpty()); assertEquals(table.getColumns().size(), 1); @@ -309,6 +335,7 @@ public void testReferencedTablesWithViews() TableInfo table = tables.get(0); assertThat(table.getCatalog()).isEqualTo("tpch"); assertThat(table.getSchema()).isEqualTo("tiny"); + assertThat(table.getTable()).isEqualTo("nation"); assertThat(table.getAuthorization()).isEqualTo("user"); assertThat(table.isDirectlyReferenced()).isFalse(); assertThat(table.getFilters()).isEmpty(); @@ -321,6 +348,44 @@ public void testReferencedTablesWithViews() table = tables.get(1); assertThat(table.getCatalog()).isEqualTo("mock"); assertThat(table.getSchema()).isEqualTo("default"); + assertThat(table.getTable()).isEqualTo("test_view"); + assertThat(table.getAuthorization()).isEqualTo("user"); + assertThat(table.isDirectlyReferenced()).isTrue(); + assertThat(table.getFilters()).isEmpty(); + assertThat(table.getColumns()).hasSize(1); + + column = table.getColumns().get(0); + assertThat(column.getColumn()).isEqualTo("test_column"); + assertThat(column.getMasks()).isEmpty(); + } + + @Test + public void testReferencedTablesWithMaterializedViews() + throws Exception + { + runQueryAndWaitForEvents("SELECT test_column FROM mock.default.test_materialized_view", 2); + + QueryCompletedEvent event = getOnlyElement(generatedEvents.getQueryCompletedEvents()); + + List tables = event.getMetadata().getTables(); + assertThat(tables).hasSize(2); + TableInfo table = tables.get(0); + assertThat(table.getCatalog()).isEqualTo("tpch"); + assertThat(table.getSchema()).isEqualTo("tiny"); + assertThat(table.getTable()).isEqualTo("nation"); + assertThat(table.getAuthorization()).isEqualTo("alice"); + assertThat(table.isDirectlyReferenced()).isFalse(); + assertThat(table.getFilters()).isEmpty(); + assertThat(table.getColumns()).hasSize(1); + + ColumnInfo column = table.getColumns().get(0); + assertThat(column.getColumn()).isEqualTo("nationkey"); + assertThat(column.getMasks()).isEmpty(); + + table = tables.get(1); + assertThat(table.getCatalog()).isEqualTo("mock"); + assertThat(table.getSchema()).isEqualTo("default"); + assertThat(table.getTable()).isEqualTo("test_materialized_view"); assertThat(table.getAuthorization()).isEqualTo("user"); assertThat(table.isDirectlyReferenced()).isTrue(); assertThat(table.getFilters()).isEmpty(); @@ -342,6 +407,12 @@ public void testReferencedTablesInCreateView() assertThat(event.getIoMetadata().getOutput().get().getCatalogName()).isEqualTo("mock"); assertThat(event.getIoMetadata().getOutput().get().getSchema()).isEqualTo("default"); assertThat(event.getIoMetadata().getOutput().get().getTable()).isEqualTo("test_view"); + assertThat(event.getIoMetadata().getOutput().get().getColumns().get()) + .containsExactly( + new OutputColumnMetadata("nationkey", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "nation", "nationkey"))), + new OutputColumnMetadata("name", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "nation", "name"))), + new OutputColumnMetadata("regionkey", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "nation", "regionkey"))), + new OutputColumnMetadata("comment", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "nation", "comment")))); List tables = event.getMetadata().getTables(); assertThat(tables).hasSize(1); @@ -367,6 +438,12 @@ public void testReferencedTablesInCreateMaterializedView() assertThat(event.getIoMetadata().getOutput().get().getCatalogName()).isEqualTo("mock"); assertThat(event.getIoMetadata().getOutput().get().getSchema()).isEqualTo("default"); assertThat(event.getIoMetadata().getOutput().get().getTable()).isEqualTo("test_view"); + assertThat(event.getIoMetadata().getOutput().get().getColumns().get()) + .containsExactly( + new OutputColumnMetadata("nationkey", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "nation", "nationkey"))), + new OutputColumnMetadata("name", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "nation", "name"))), + new OutputColumnMetadata("regionkey", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "nation", "regionkey"))), + new OutputColumnMetadata("comment", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "nation", "comment")))); List tables = event.getMetadata().getTables(); assertThat(tables).hasSize(1); @@ -423,19 +500,28 @@ public void testReferencedTablesWithRowFilter() public void testReferencedTablesWithColumnMask() throws Exception { - runQueryAndWaitForEvents("SELECT * FROM mock.default.test_table_with_column_mask", 2); + runQueryAndWaitForEvents("CREATE TABLE mock.default.create_table_with_referring_mask AS SELECT * FROM mock.default.test_table_with_column_mask", 2); QueryCompletedEvent event = getOnlyElement(generatedEvents.getQueryCompletedEvents()); + assertThat(event.getIoMetadata().getOutput().get().getCatalogName()).isEqualTo("mock"); + assertThat(event.getIoMetadata().getOutput().get().getSchema()).isEqualTo("default"); + assertThat(event.getIoMetadata().getOutput().get().getTable()).isEqualTo("create_table_with_referring_mask"); + assertThat(event.getIoMetadata().getOutput().get().getColumns().get()) + .containsExactly( + new OutputColumnMetadata("test_varchar", ImmutableSet.of(new ColumnDetail("mock", "default", "test_table_with_column_mask", "test_varchar"))), + new OutputColumnMetadata("test_bigint", ImmutableSet.of(new ColumnDetail("mock", "default", "test_table_with_column_mask", "test_bigint")))); + List tables = event.getMetadata().getTables(); assertThat(tables).hasSize(2); TableInfo table = tables.get(0); assertThat(table.getCatalog()).isEqualTo("tpch"); assertThat(table.getSchema()).isEqualTo("tiny"); + assertThat(table.getTable()).isEqualTo("orders"); assertThat(table.getAuthorization()).isEqualTo("user"); assertThat(table.isDirectlyReferenced()).isFalse(); - assertThat(table.getFilters()).hasSize(0); + assertThat(table.getFilters()).isEmpty(); assertThat(table.getColumns()).hasSize(1); ColumnInfo column = table.getColumns().get(0); @@ -445,14 +531,19 @@ public void testReferencedTablesWithColumnMask() table = tables.get(1); assertThat(table.getCatalog()).isEqualTo("mock"); assertThat(table.getSchema()).isEqualTo("default"); + assertThat(table.getTable()).isEqualTo("test_table_with_column_mask"); assertThat(table.getAuthorization()).isEqualTo("user"); assertThat(table.isDirectlyReferenced()).isTrue(); assertThat(table.getFilters()).isEmpty(); - assertThat(table.getColumns()).hasSize(1); + assertThat(table.getColumns()).hasSize(2); column = table.getColumns().get(0); assertThat(column.getColumn()).isEqualTo("test_varchar"); assertThat(column.getMasks()).hasSize(1); + + column = table.getColumns().get(1); + assertThat(column.getColumn()).isEqualTo("test_bigint"); + assertThat(column.getMasks()).isEmpty(); } @Test @@ -599,4 +690,339 @@ public void testOutputStats() assertEquals(statistics.getStageGcStatistics(), queryStats.getStageGcStatistics()); assertEquals(statistics.getCompletedSplits(), queryStats.getCompletedDrivers()); } + + @Test + public void testOutputColumnsForSelect() + throws Exception + { + assertColumnLineage( + "SELECT name as test_varchar, nationkey as test_bigint FROM nation", + new OutputColumnMetadata("test_varchar", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "nation", "name"))), + new OutputColumnMetadata("test_bigint", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "nation", "nationkey")))); + } + + @Test + public void testOutputColumnsForSelectWithConstantExpression() + throws Exception + { + assertColumnLineage( + "SELECT 'Trino' as test_varchar, nationkey as test_bigint FROM nation", + new OutputColumnMetadata("test_varchar", ImmutableSet.of()), + new OutputColumnMetadata("test_bigint", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "nation", "nationkey")))); + } + + @Test + public void testOutputColumnsForCreateTableAsSelectAll() + throws Exception + { + runQueryAndWaitForEvents("CREATE TABLE mock.default.create_new_table AS SELECT * FROM nation", 2); + QueryCompletedEvent event = getOnlyElement(generatedEvents.getQueryCompletedEvents()); + assertThat(event.getIoMetadata().getOutput().get().getColumns().get()) + .containsExactly( + new OutputColumnMetadata("nationkey", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "nation", "nationkey"))), + new OutputColumnMetadata("name", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "nation", "name"))), + new OutputColumnMetadata("regionkey", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "nation", "regionkey"))), + new OutputColumnMetadata("comment", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "nation", "comment")))); + } + + @Test + public void testOutputColumnsForCreateTableAsSelectAllFromView() + throws Exception + { + runQueryAndWaitForEvents("CREATE TABLE mock.default.create_new_table AS SELECT * FROM mock.default.test_view", 2); + QueryCompletedEvent event = getOnlyElement(generatedEvents.getQueryCompletedEvents()); + assertThat(event.getIoMetadata().getOutput().get().getColumns().get()) + .containsExactly( + new OutputColumnMetadata("test_column", ImmutableSet.of(new ColumnDetail("mock", "default", "test_view", "test_column")))); + } + + @Test + public void testOutputColumnsForCreateTableAsSelectAllFromMaterializedView() + throws Exception + { + runQueryAndWaitForEvents("CREATE TABLE mock.default.create_new_table AS SELECT * FROM mock.default.test_materialized_view", 2); + QueryCompletedEvent event = getOnlyElement(generatedEvents.getQueryCompletedEvents()); + assertThat(event.getIoMetadata().getOutput().get().getColumns().get()) + .containsExactly( + new OutputColumnMetadata("test_column", ImmutableSet.of(new ColumnDetail("mock", "default", "test_materialized_view", "test_column")))); + } + + @Test + public void testOutputColumnsForCreateTableAsSelectWithAliasedColumn() + throws Exception + { + runQueryAndWaitForEvents("CREATE TABLE mock.default.create_new_table(aliased_bigint, aliased_varchar) AS SELECT nationkey as keynation, concat(name, comment) FROM nation", 2); + QueryCompletedEvent event = getOnlyElement(generatedEvents.getQueryCompletedEvents()); + assertThat(event.getIoMetadata().getOutput().get().getColumns().get()) + .containsExactly( + new OutputColumnMetadata("aliased_bigint", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "nation", "nationkey"))), + new OutputColumnMetadata("aliased_varchar", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "nation", "name"), new ColumnDetail("tpch", "tiny", "nation", "comment")))); + } + + @Test + public void testOutputColumnsWithClause() + throws Exception + { + assertColumnLineage( + "WITH w AS (SELECT * FROM nation) SELECT name as test_varchar, nationkey as test_bigint FROM w", + new OutputColumnMetadata("test_varchar", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "nation", "name"))), + new OutputColumnMetadata("test_bigint", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "nation", "nationkey")))); + } + + @Test + public void testOutputColumnsWithWhere() + throws Exception + { + assertColumnLineage( + "SELECT name as test_varchar, nationkey as test_bigint FROM nation WHERE regionkey IS NULL", + new OutputColumnMetadata("test_varchar", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "nation", "name"))), + new OutputColumnMetadata("test_bigint", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "nation", "nationkey")))); + } + + @Test + public void testOutputColumnsWithIfExpression() + throws Exception + { + assertColumnLineage( + "SELECT if (regionkey > 100, name, comment) as test_varchar, nationkey as test_bigint FROM nation", + new OutputColumnMetadata( + "test_varchar", + ImmutableSet.of( + new ColumnDetail("tpch", "tiny", "nation", "regionkey"), + new ColumnDetail("tpch", "tiny", "nation", "name"), + new ColumnDetail("tpch", "tiny", "nation", "comment"))), + new OutputColumnMetadata("test_bigint", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "nation", "nationkey")))); + } + + @Test + public void testOutputColumnsWithCaseExpression() + throws Exception + { + assertColumnLineage( + "SELECT CASE WHEN regionkey = 100 THEN name WHEN regionkey = 1000 then comment ELSE CAST(regionkey AS VARCHAR) END as test_varchar, nationkey as test_bigint FROM nation", + new OutputColumnMetadata( + "test_varchar", + ImmutableSet.of( + new ColumnDetail("tpch", "tiny", "nation", "regionkey"), + new ColumnDetail("tpch", "tiny", "nation", "name"), + new ColumnDetail("tpch", "tiny", "nation", "comment"))), + new OutputColumnMetadata("test_bigint", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "nation", "nationkey")))); + } + + @Test + public void testOutputColumnsWithLimit() + throws Exception + { + assertColumnLineage( + "SELECT name as test_varchar, nationkey as test_bigint FROM nation LIMIT 100", + new OutputColumnMetadata("test_varchar", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "nation", "name"))), + new OutputColumnMetadata("test_bigint", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "nation", "nationkey")))); + } + + @Test + public void testOutputColumnsWithOrderBy() + throws Exception + { + assertColumnLineage( + "SELECT name as test_varchar, nationkey as test_bigint FROM nation ORDER BY comment", + new OutputColumnMetadata("test_varchar", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "nation", "name"))), + new OutputColumnMetadata("test_bigint", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "nation", "nationkey")))); + } + + @Test + public void testOutputColumnsWithAggregation() + throws Exception + { + assertColumnLineage( + "SELECT max(orderstatus) as test_varchar, sum(totalprice) as test_bigint FROM orders GROUP BY custkey", + new OutputColumnMetadata("test_varchar", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "orders", "orderstatus"))), + new OutputColumnMetadata("test_bigint", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "orders", "totalprice")))); + } + + @Test + public void testOutputColumnsWithAggregationWithFilter() + throws Exception + { + assertColumnLineage( + "SELECT max(orderstatus) FILTER(WHERE orderdate > DATE '2000-01-01') as test_varchar, sum(totalprice) as test_bigint FROM orders GROUP BY custkey", + new OutputColumnMetadata( + "test_varchar", + ImmutableSet.of( + new ColumnDetail("tpch", "tiny", "orders", "orderstatus"), + new ColumnDetail("tpch", "tiny", "orders", "orderdate"))), + new OutputColumnMetadata("test_bigint", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "orders", "totalprice")))); + } + + @Test + public void testOutputColumnsWithAggregationAndHaving() + throws Exception + { + assertColumnLineage( + "SELECT min(orderstatus) as test_varchar, sum(totalprice) as test_bigint FROM orders GROUP BY custkey HAVING min(orderdate) > DATE '2000-01-01'", + new OutputColumnMetadata("test_varchar", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "orders", "orderstatus"))), + new OutputColumnMetadata("test_bigint", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "orders", "totalprice")))); + } + + @Test + public void testOutputColumnsWithCountAll() + throws Exception + { + assertColumnLineage( + "SELECT orderstatus as test_varchar, count(*) as test_bigint FROM orders GROUP BY orderstatus", + new OutputColumnMetadata("test_varchar", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "orders", "orderstatus"))), + new OutputColumnMetadata("test_bigint", ImmutableSet.of())); + } + + @Test + public void testOutputColumnsWithWindowFunction() + throws Exception + { + assertColumnLineage( + "SELECT orderstatus as test_varchar, avg(totalprice) OVER (PARTITION BY custkey ORDER BY orderdate ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS test_bigint FROM orders", + new OutputColumnMetadata("test_varchar", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "orders", "orderstatus"))), + new OutputColumnMetadata( + "test_bigint", + ImmutableSet.of( + new ColumnDetail("tpch", "tiny", "orders", "totalprice"), + new ColumnDetail("tpch", "tiny", "orders", "custkey"), + new ColumnDetail("tpch", "tiny", "orders", "orderdate")))); + } + + @Test + public void testOutputColumnsWithPartialWindowClause() + throws Exception + { + assertColumnLineage( + "SELECT orderstatus as test_varchar, sum(totalprice) OVER (w ORDER BY orderdate ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS test_bigint FROM orders WINDOW w AS (PARTITION BY custkey)", + new OutputColumnMetadata("test_varchar", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "orders", "orderstatus"))), + new OutputColumnMetadata( + "test_bigint", + ImmutableSet.of( + new ColumnDetail("tpch", "tiny", "orders", "totalprice"), + new ColumnDetail("tpch", "tiny", "orders", "orderdate")))); + } + + @Test + public void testOutputColumnsWithWindowClause() + throws Exception + { + assertColumnLineage( + "SELECT orderstatus as test_varchar, sum(totalprice) OVER w AS test_bigint FROM orders WINDOW w AS (PARTITION BY custkey ORDER BY orderdate ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)", + new OutputColumnMetadata("test_varchar", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "orders", "orderstatus"))), + new OutputColumnMetadata("test_bigint", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "orders", "totalprice")))); + } + + @Test + public void testOutputColumnsWithUnCorrelatedQueries() + throws Exception + { + assertColumnLineage( + "SELECT orderstatus as test_varchar, (SELECT nationkey FROM nation LIMIT 1) as test_bigint FROM orders", + new OutputColumnMetadata("test_varchar", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "orders", "orderstatus"))), + new OutputColumnMetadata("test_bigint", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "nation", "nationkey")))); + } + + @Test + public void testOutputColumnsWithCorrelatedQueries() + throws Exception + { + assertColumnLineage( + "SELECT name as test_varchar, (SELECT sum(acctbal) FROM supplier WHERE supplier.nationkey=nation.nationkey) as test_bigint FROM nation", + new OutputColumnMetadata("test_varchar", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "nation", "name"))), + new OutputColumnMetadata("test_bigint", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "supplier", "acctbal")))); + } + + @Test + public void testOutputColumnsForInsertingSingleColumn() + throws Exception + { + runQueryAndWaitForEvents("INSERT INTO mock.default.table_for_output(test_bigint) SELECT nationkey + 1 as test_bigint FROM nation", 2); + QueryCompletedEvent event = getOnlyElement(generatedEvents.getQueryCompletedEvents()); + assertThat(event.getIoMetadata().getOutput().get().getColumns().get()) + .containsExactly(new OutputColumnMetadata("test_bigint", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "nation", "nationkey")))); + } + + @Test + public void testOutputColumnsForInsertingAliasedColumn() + throws Exception + { + runQueryAndWaitForEvents("INSERT INTO mock.default.table_for_output(test_varchar, test_bigint) SELECT name as aliased_name, nationkey as aliased_varchar FROM nation", 2); + QueryCompletedEvent event = getOnlyElement(generatedEvents.getQueryCompletedEvents()); + assertThat(event.getIoMetadata().getOutput().get().getColumns().get()) + .containsExactly( + new OutputColumnMetadata("test_varchar", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "nation", "name"))), + new OutputColumnMetadata("test_bigint", ImmutableSet.of(new ColumnDetail("tpch", "tiny", "nation", "nationkey")))); + } + + @Test + public void testOutputColumnsForUpdatingAllColumns() + throws Exception + { + runQueryAndWaitForEvents("UPDATE mock.default.table_for_output SET test_varchar = 'reset', test_bigint = 1", 2); + QueryCompletedEvent event = getOnlyElement(generatedEvents.getQueryCompletedEvents()); + assertThat(event.getIoMetadata().getOutput().get().getColumns().get()) + .containsExactly(new OutputColumnMetadata("test_varchar", ImmutableSet.of()), new OutputColumnMetadata("test_bigint", ImmutableSet.of())); + } + + @Test + public void testOutputColumnsForUpdatingSingleColumn() + throws Exception + { + runQueryAndWaitForEvents("UPDATE mock.default.table_for_output SET test_varchar = 're-reset' WHERE test_bigint = 1", 2); + QueryCompletedEvent event = getOnlyElement(generatedEvents.getQueryCompletedEvents()); + assertThat(event.getIoMetadata().getOutput().get().getColumns().get()) + .containsExactly(new OutputColumnMetadata("test_varchar", ImmutableSet.of())); + } + + @Test + public void testCreateTable() + throws Exception + { + runQueryAndWaitForEvents("CREATE TABLE mock.default.create_simple_table (test_column BIGINT)", 2); + QueryCompletedEvent event = getOnlyElement(generatedEvents.getQueryCompletedEvents()); + assertThat(event.getIoMetadata().getOutput().get().getCatalogName()).isEqualTo("mock"); + assertThat(event.getIoMetadata().getOutput().get().getSchema()).isEqualTo("default"); + assertThat(event.getIoMetadata().getOutput().get().getTable()).isEqualTo("create_simple_table"); + assertThat(event.getIoMetadata().getOutput().get().getColumns().get()) + .containsExactly(new OutputColumnMetadata("test_column", ImmutableSet.of())); + } + + @Test + public void testCreateTableLike() + throws Exception + { + runQueryAndWaitForEvents("CREATE TABLE mock.default.create_simple_table (test_column BIGINT, LIKE mock.default.test_table)", 2); + QueryCompletedEvent event = getOnlyElement(generatedEvents.getQueryCompletedEvents()); + assertThat(event.getIoMetadata().getOutput().get().getCatalogName()).isEqualTo("mock"); + assertThat(event.getIoMetadata().getOutput().get().getSchema()).isEqualTo("default"); + assertThat(event.getIoMetadata().getOutput().get().getTable()).isEqualTo("create_simple_table"); + assertThat(event.getIoMetadata().getOutput().get().getColumns().get()) + .containsExactly( + new OutputColumnMetadata("test_column", ImmutableSet.of()), + new OutputColumnMetadata("test_varchar", ImmutableSet.of()), + new OutputColumnMetadata("test_bigint", ImmutableSet.of())); + } + + private void assertColumnLineage(String baseQuery, OutputColumnMetadata... outputColumnMetadata) + throws Exception + { + runQueryAndWaitForEvents("CREATE TABLE mock.default.create_new_table AS " + baseQuery, 2); + assertColumnMetadata(outputColumnMetadata); + + runQueryAndWaitForEvents("CREATE VIEW mock.default.create_new_view AS " + baseQuery, 2); + assertColumnMetadata(outputColumnMetadata); + + runQueryAndWaitForEvents("CREATE VIEW mock.default.create_new_materialized_view AS " + baseQuery, 2); + assertColumnMetadata(outputColumnMetadata); + + runQueryAndWaitForEvents("INSERT INTO mock.default.table_for_output(test_varchar, test_bigint) " + baseQuery, 2); + assertColumnMetadata(outputColumnMetadata); + } + + private void assertColumnMetadata(OutputColumnMetadata... outputColumnMetadata) + { + QueryCompletedEvent event = getOnlyElement(generatedEvents.getQueryCompletedEvents()); + assertThat(event.getIoMetadata().getOutput().get().getColumns().get()) + .containsExactly(outputColumnMetadata); + } } diff --git a/testing/trino-tests/src/test/java/io/trino/memory/TestClusterMemoryLeakDetector.java b/testing/trino-tests/src/test/java/io/trino/memory/TestClusterMemoryLeakDetector.java index 6bffe0475d89..3d89b00929a6 100644 --- a/testing/trino-tests/src/test/java/io/trino/memory/TestClusterMemoryLeakDetector.java +++ b/testing/trino-tests/src/test/java/io/trino/memory/TestClusterMemoryLeakDetector.java @@ -35,6 +35,7 @@ import static io.trino.execution.QueryState.RUNNING; import static io.trino.memory.LocalMemoryManager.GENERAL_POOL; import static io.trino.operator.BlockedReason.WAITING_FOR_MEMORY; +import static java.util.concurrent.TimeUnit.MINUTES; import static org.testng.Assert.assertEquals; @Test @@ -82,9 +83,9 @@ private static BasicQueryInfo createQueryInfo(String queryId, QueryState state) new BasicQueryStats( DateTime.parse("1991-09-06T05:00-05:30"), DateTime.parse("1991-09-06T05:01-05:30"), - Duration.valueOf("8m"), - Duration.valueOf("7m"), - Duration.valueOf("34m"), + new Duration(8, MINUTES), + new Duration(7, MINUTES), + new Duration(34, MINUTES), 13, 14, 15, @@ -97,8 +98,8 @@ private static BasicQueryInfo createQueryInfo(String queryId, QueryState state) DataSize.valueOf("24GB"), DataSize.valueOf("25GB"), DataSize.valueOf("26GB"), - Duration.valueOf("23m"), - Duration.valueOf("24m"), + new Duration(23, MINUTES), + new Duration(24, MINUTES), true, ImmutableSet.of(WAITING_FOR_MEMORY), OptionalDouble.of(20)), diff --git a/testing/trino-tests/src/test/java/io/trino/tests/TestDistributedSpilledQueries.java b/testing/trino-tests/src/test/java/io/trino/tests/TestDistributedSpilledQueries.java index 693c2feee35b..c19ef00cc73c 100644 --- a/testing/trino-tests/src/test/java/io/trino/tests/TestDistributedSpilledQueries.java +++ b/testing/trino-tests/src/test/java/io/trino/tests/TestDistributedSpilledQueries.java @@ -72,7 +72,8 @@ public static DistributedQueryRunner createSpillingQueryRunner() } } - @Test + // The spilling does not happen deterministically. TODO improve query and configuration so that it does. + @Test(invocationCount = 10, successPercentage = 20) public void testExplainAnalyzeReportSpilledDataSize() { assertThat((String) computeActual("EXPLAIN ANALYZE SELECT sum(custkey) OVER (PARTITION BY orderkey) FROM orders").getOnlyValue()) diff --git a/testing/trino-tests/src/test/java/io/trino/tests/TestInformationSchemaConnector.java b/testing/trino-tests/src/test/java/io/trino/tests/TestInformationSchemaConnector.java index 9ede07fcd826..894b0679fd7c 100644 --- a/testing/trino-tests/src/test/java/io/trino/tests/TestInformationSchemaConnector.java +++ b/testing/trino-tests/src/test/java/io/trino/tests/TestInformationSchemaConnector.java @@ -185,7 +185,8 @@ public void testMetadataCalls() "SELECT count(*) from test_catalog.information_schema.tables WHERE table_name = 'test_table1'", "VALUES 2", new MetadataCallsCount() - .withListSchemasCount(1)); + .withListSchemasCount(1) + .withGetTableHandleCount(2)); assertMetadataCalls( "SELECT count(*) from test_catalog.information_schema.tables WHERE table_name LIKE 'test_t_ble1'", "VALUES 2", @@ -196,12 +197,14 @@ public void testMetadataCalls() "SELECT count(*) from test_catalog.information_schema.tables WHERE table_name LIKE 'test_t_ble1' AND table_name IN ('test_table1', 'test_table2')", "VALUES 2", new MetadataCallsCount() - .withListSchemasCount(1)); + .withListSchemasCount(1) + .withGetTableHandleCount(4)); assertMetadataCalls( "SELECT count(*) from test_catalog.information_schema.columns WHERE table_schema = 'test_schema1' AND table_name = 'test_table1'", "VALUES 100", new MetadataCallsCount() .withListTablesCount(1) + .withGetTableHandleCount(1) .withGetColumnsCount(1)); assertMetadataCalls( "SELECT count(*) from test_catalog.information_schema.columns WHERE table_catalog = 'wrong'", @@ -211,12 +214,14 @@ public void testMetadataCalls() "SELECT count(*) from test_catalog.information_schema.columns WHERE table_catalog = 'test_catalog' AND table_schema = 'wrong_schema1' AND table_name = 'test_table1'", "VALUES 0", new MetadataCallsCount() - .withListTablesCount(1)); + .withListTablesCount(1) + .withGetTableHandleCount(1)); assertMetadataCalls( "SELECT count(*) from test_catalog.information_schema.columns WHERE table_catalog IN ('wrong', 'test_catalog') AND table_schema = 'wrong_schema1' AND table_name = 'test_table1'", "VALUES 0", new MetadataCallsCount() - .withListTablesCount(1)); + .withListTablesCount(1) + .withGetTableHandleCount(1)); assertMetadataCalls( "SELECT count(*) FROM (SELECT * from test_catalog.information_schema.columns LIMIT 1)", "VALUES 1", diff --git a/testing/trino-tests/src/test/java/io/trino/tests/TestMockConnector.java b/testing/trino-tests/src/test/java/io/trino/tests/TestMockConnector.java new file mode 100644 index 000000000000..e71c0fecb174 --- /dev/null +++ b/testing/trino-tests/src/test/java/io/trino/tests/TestMockConnector.java @@ -0,0 +1,104 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.trino.tests; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.trino.connector.MockConnectorFactory; +import io.trino.connector.MockConnectorPlugin; +import io.trino.connector.MockConnectorTableHandle; +import io.trino.plugin.tpch.TpchPlugin; +import io.trino.spi.connector.ColumnMetadata; +import io.trino.spi.connector.ConnectorMaterializedViewDefinition; +import io.trino.spi.connector.ConnectorMaterializedViewDefinition.Column; +import io.trino.spi.connector.SchemaTableName; +import io.trino.testing.AbstractTestQueryFramework; +import io.trino.testing.DistributedQueryRunner; +import io.trino.testing.QueryRunner; +import org.testng.annotations.Test; + +import java.util.Optional; + +import static io.trino.spi.type.BigintType.BIGINT; +import static io.trino.testing.TestingSession.testSessionBuilder; + +public class TestMockConnector + extends AbstractTestQueryFramework +{ + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + DistributedQueryRunner queryRunner = DistributedQueryRunner.builder(testSessionBuilder().build()).build(); + + queryRunner.installPlugin(new TpchPlugin()); + queryRunner.createCatalog("tpch", "tpch"); + queryRunner.installPlugin( + new MockConnectorPlugin( + MockConnectorFactory.builder() + .withListSchemaNames(connectionSession -> ImmutableList.of("default")) + .withGetColumns(schemaTableName -> ImmutableList.of(new ColumnMetadata("nationkey", BIGINT))) + .withGetTableHandle((session, tableName) -> new MockConnectorTableHandle(tableName)) + .withGetMaterializedViews((session, schemaTablePrefix) -> ImmutableMap.of( + new SchemaTableName("default", "test_materialized_view"), + new ConnectorMaterializedViewDefinition( + "SELECT nationkey FROM mock.default.test_table", + Optional.of("test_storage"), + Optional.of("mock"), + Optional.of("default"), + ImmutableList.of(new Column("nationkey", BIGINT.getTypeId())), + Optional.empty(), + "alice", + ImmutableMap.of()))) + .build())); + queryRunner.createCatalog("mock", "mock"); + return queryRunner; + } + + @Test + public void testCreateSchema() + { + assertUpdate("CREATE SCHEMA mock.new_schema"); + } + + @Test + public void testDropSchema() + { + assertUpdate("DROP SCHEMA mock.default"); + } + + @Test + public void testRenameSchema() + { + assertUpdate("ALTER SCHEMA mock.default RENAME to renamed"); + } + + @Test + public void testCreateMaterializedView() + { + assertUpdate("CREATE MATERIALIZED VIEW mock.default.materialized_view AS SELECT * FROM tpch.tiny.nation"); + } + + @Test + public void testRefreshMaterializedView() + { + assertUpdate("REFRESH MATERIALIZED VIEW mock.default.test_materialized_view", 0); + } + + @Test + public void testDropMaterializedView() + { + assertUpdate("DROP MATERIALIZED VIEW mock.default.test_materialized_view"); + } +}