From b0a4ced61ad0e1aec196455e6edb8f7f6b041aee Mon Sep 17 00:00:00 2001 From: Paul Rogers Date: Sat, 14 Jan 2023 14:36:09 -0800 Subject: [PATCH 01/43] Snapshot --- extensions-contrib/grpc-query/pom.xml | 175 ++++++++++++++++++ .../druid/grpc/guice/GrpcQueryModule.java | 16 ++ .../apache/druid/grpc/server/QueryServer.java | 76 ++++++++ .../grpc-query/src/main/proto/query.proto | 89 +++++++++ ...rg.apache.druid.initialization.DruidModule | 16 ++ 5 files changed, 372 insertions(+) create mode 100644 extensions-contrib/grpc-query/pom.xml create mode 100644 extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/guice/GrpcQueryModule.java create mode 100644 extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryServer.java create mode 100644 extensions-contrib/grpc-query/src/main/proto/query.proto create mode 100644 extensions-contrib/grpc-query/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule diff --git a/extensions-contrib/grpc-query/pom.xml b/extensions-contrib/grpc-query/pom.xml new file mode 100644 index 000000000000..1f7c586b2590 --- /dev/null +++ b/extensions-contrib/grpc-query/pom.xml @@ -0,0 +1,175 @@ + + + + + 4.0.0 + org.apache.druid.extensions.contrib + grpc-query + grpc-query + grpc-query + + + org.apache.druid + druid + 26.0.0-SNAPSHOT + ../../pom.xml + + + + + org.apache.druid + druid-core + ${project.parent.version} + provided + + + io.grpc + grpc-netty-shaded + 1.52.0 + runtime + + + io.grpc + grpc-protobuf + 1.52.0 + + + io.grpc + grpc-stub + 1.52.0 + + + + com.fasterxml.jackson.module + jackson-module-guice + provided + + + com.google.inject + guice + provided + + + com.fasterxml.jackson.core + jackson-databind + provided + + + com.fasterxml.jackson.core + jackson-core + provided + + + com.google.inject.extensions + guice-multibindings + provided + + + com.google.guava + guava + provided + + + com.google.code.findbugs + jsr305 + provided + + + + + org.apache.druid + druid-core + ${project.parent.version} + test-jar + test + + + junit + junit + test + + + org.apache.druid + druid-server + ${project.parent.version} + test + + + org.apache.druid + druid-processing + ${project.parent.version} + test + + + + + + + kr.motd.maven + os-maven-plugin + 1.6.2 + + + + + org.xolstice.maven.plugins + protobuf-maven-plugin + 0.6.1 + + com.google.protobuf:protoc:3.21.7:exe:${os.detected.classifier} + grpc-java + io.grpc:protoc-gen-grpc-java:1.52.0:exe:${os.detected.classifier} + + + + + compile + compile-custom + + + + + + org.apache.maven.plugins + maven-assembly-plugin + + false + + jar-with-dependencies + + + + true + + + + + + make-assembly + package + + single + + + + + + + diff --git a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/guice/GrpcQueryModule.java b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/guice/GrpcQueryModule.java new file mode 100644 index 000000000000..78aafd848cf9 --- /dev/null +++ b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/guice/GrpcQueryModule.java @@ -0,0 +1,16 @@ +package org.apache.druid.grpc.guice; + +import com.google.inject.Binder; +import org.apache.druid.initialization.DruidModule; + +public class GrpcQueryModule implements DruidModule +{ + + @Override + public void configure(Binder binder) + { + // TODO Auto-generated method stub + + } + +} diff --git a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryServer.java b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryServer.java new file mode 100644 index 000000000000..b2dc3f361761 --- /dev/null +++ b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryServer.java @@ -0,0 +1,76 @@ +package org.apache.druid.grpc.server; + +import io.grpc.Grpc; +import io.grpc.InsecureServerCredentials; +import io.grpc.Server; +import io.grpc.stub.StreamObserver; +import java.io.IOException; +import java.util.concurrent.TimeUnit; +import java.util.logging.Logger; +import org.druid.grpc.proto.QueryGrpc; +import org.druid.grpc.proto.QueryOuterClass.QueryRequest; +import org.druid.grpc.proto.QueryOuterClass.QueryResponse; + + +public class QueryServer +{ + private static final Logger logger = Logger.getLogger(QueryServer.class.getName()); + + private Server server; + + private void start() throws IOException { + /* The port on which the server should run */ + int port = 50051; + server = Grpc.newServerBuilderForPort(port, InsecureServerCredentials.create()) + .addService(new QueryImpl()) + .build() + .start(); + logger.info("Server started, listening on " + port); + Runtime.getRuntime().addShutdownHook(new Thread() { + @Override + public void run() { + // Use stderr here since the logger may have been reset by its JVM shutdown hook. + System.err.println("*** shutting down gRPC server since JVM is shutting down"); + try { + QueryServer.this.stop(); + } catch (InterruptedException e) { + e.printStackTrace(System.err); + } + System.err.println("*** server shut down"); + } + }); + } + + private void stop() throws InterruptedException { + if (server != null) { + server.shutdown().awaitTermination(30, TimeUnit.SECONDS); + } + } + + /** + * Await termination on the main thread since the grpc library uses daemon threads. + */ + private void blockUntilShutdown() throws InterruptedException { + if (server != null) { + server.awaitTermination(); + } + } + + /** + * Main launches the server from the command line. + */ + public static void main(String[] args) throws IOException, InterruptedException { + final QueryServer server = new QueryServer(); + server.start(); + server.blockUntilShutdown(); + } + + static class QueryImpl extends QueryGrpc.QueryImplBase { + + @Override + public void submitQuery(QueryRequest request, StreamObserver responseObserver) + { + } + } + +} diff --git a/extensions-contrib/grpc-query/src/main/proto/query.proto b/extensions-contrib/grpc-query/src/main/proto/query.proto new file mode 100644 index 000000000000..e25cbb79e85c --- /dev/null +++ b/extensions-contrib/grpc-query/src/main/proto/query.proto @@ -0,0 +1,89 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +syntax = "proto3"; + +option java_package = "org.druid.grpc.proto"; + +package druidGrpc; + +service Query { + rpc SubmitQuery (QueryRequest) returns (QueryResponse) {} +} + +enum QueryResultFormat { + CSV = 0; + JSON_OBJECT = 1; + JSON_ARRAY = 2; + JSON_OBJECT_LINES = 3; + JSON_ARRAY_LINES = 4; + PROTOBUF_INLINE = 5; + PROTOBUF_RESPONSE = 6; +} + +message QueryParameter { + oneof value { + bool nullValue = 1; + string stringValue = 2; + sint64 longValue = 3; + double doubleValue = 4; + StringArray arrayValue = 5; + } +} + +message StringArray { + repeated string value = 1; +} + +message QueryRequest { + string query = 1; + QueryResultFormat resultFormat = 2; + map context = 3; + repeated QueryParameter parameters = 4; + optional string responseMessage = 5; +} + +enum QueryStatus { + OK = 0; + UNAUTHORIZED = 1; + INVALID_SQL = 2; + RUNTIME_ERROR = 3; +} + +enum DruidType { + STRING = 0; + LONG = 1; + DOUBLE = 2; + FLOAT = 3; + STRING_ARRAY = 4; + LONG_ARRAY = 5; + DOUBLE_ARRAY = 6; + FLOAT_ARRAY = 7; + COMPLEX = 8; +} + +message ColumnSchema { + string name = 1; + string sqlType = 2; + DruidType druidType = 3; +} + +message QueryResponse { + string queryId = 1; + QueryStatus status = 2; + optional string errorMessage = 3; + repeated ColumnSchema columns = 4; + optional bytes data = 5; +} \ No newline at end of file diff --git a/extensions-contrib/grpc-query/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule b/extensions-contrib/grpc-query/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule new file mode 100644 index 000000000000..d1e9a4fc482a --- /dev/null +++ b/extensions-contrib/grpc-query/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule @@ -0,0 +1,16 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +org.apache.druid.grpc.GrpcQueryModule From f0f3c673a032dade75cbb0b89566ef61a5b38f87 Mon Sep 17 00:00:00 2001 From: Paul Rogers Date: Sun, 15 Jan 2023 14:18:05 -0800 Subject: [PATCH 02/43] Snapshot --- extensions-contrib/grpc-query/pom.xml | 30 +-- .../druid/grpc/guice/GrpcQueryModule.java | 22 +- .../apache/druid/grpc/server/QueryDriver.java | 196 ++++++++++++++++++ .../apache/druid/grpc/server/QueryServer.java | 63 ++++-- .../grpc-query/src/main/proto/query.proto | 2 +- .../org/apache/druid/grpc/GrpcQueryTest.java | 94 +++++++++ pom.xml | 1 + .../druid/server/QueryResultPusher.java | 2 +- .../druid/sql/SqlPlanningException.java | 2 - .../apache/druid/sql/http/SqlResource.java | 16 +- 10 files changed, 380 insertions(+), 48 deletions(-) create mode 100644 extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriver.java create mode 100644 extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTest.java diff --git a/extensions-contrib/grpc-query/pom.xml b/extensions-contrib/grpc-query/pom.xml index 1f7c586b2590..4db73fce5bc6 100644 --- a/extensions-contrib/grpc-query/pom.xml +++ b/extensions-contrib/grpc-query/pom.xml @@ -38,6 +38,24 @@ druid-core ${project.parent.version} provided + + + org.apache.druid + druid-server + ${project.parent.version} + provided + + + org.apache.druid + druid-processing + ${project.parent.version} + provided + + + org.apache.druid + druid-sql + ${project.parent.version} + provided io.grpc @@ -105,18 +123,6 @@ junit test - - org.apache.druid - druid-server - ${project.parent.version} - test - - - org.apache.druid - druid-processing - ${project.parent.version} - test - diff --git a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/guice/GrpcQueryModule.java b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/guice/GrpcQueryModule.java index 78aafd848cf9..d5cae6eb86cd 100644 --- a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/guice/GrpcQueryModule.java +++ b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/guice/GrpcQueryModule.java @@ -1,3 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package org.apache.druid.grpc.guice; import com.google.inject.Binder; @@ -5,12 +24,9 @@ public class GrpcQueryModule implements DruidModule { - @Override public void configure(Binder binder) { // TODO Auto-generated method stub - } - } diff --git a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriver.java b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriver.java new file mode 100644 index 000000000000..802590c85ab5 --- /dev/null +++ b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriver.java @@ -0,0 +1,196 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.grpc.server; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableMap; +import com.google.inject.Inject; +import com.google.protobuf.ByteString; +import org.apache.calcite.avatica.SqlType; +import org.apache.druid.guice.annotations.Json; +import org.apache.druid.guice.annotations.NativeQuery; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.server.security.AuthConfig; +import org.apache.druid.server.security.AuthenticationResult; +import org.apache.druid.server.security.ForbiddenException; +import org.apache.druid.sql.DirectStatement; +import org.apache.druid.sql.SqlPlanningException; +import org.apache.druid.sql.SqlQueryPlus; +import org.apache.druid.sql.SqlRowTransformer; +import org.apache.druid.sql.SqlStatementFactory; +import org.apache.druid.sql.DirectStatement.ResultSet; +import org.apache.druid.sql.http.ResultFormat; +import org.apache.druid.sql.http.SqlParameter; +import org.druid.grpc.proto.QueryOuterClass.QueryParameter; +import org.druid.grpc.proto.QueryOuterClass.QueryRequest; +import org.druid.grpc.proto.QueryOuterClass.QueryResponse; +import org.druid.grpc.proto.QueryOuterClass.QueryResultFormat; +import org.druid.grpc.proto.QueryOuterClass.QueryStatus; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +public class QueryDriver +{ + private final ObjectMapper jsonMapper; + private final SqlStatementFactory sqlStatementFactory; + + @Inject + public QueryDriver( + final @Json ObjectMapper jsonMapper, + final @NativeQuery SqlStatementFactory sqlStatementFactory + ) + { + this.jsonMapper = Preconditions.checkNotNull(jsonMapper, "jsonMapper"); + this.sqlStatementFactory = Preconditions.checkNotNull(sqlStatementFactory, "sqlStatementFactory"); + } + + public QueryResponse submitQuery(QueryRequest request) + { + final SqlQueryPlus queryPlus = translateQuery(request); + final DirectStatement stmt = sqlStatementFactory.directStatement(queryPlus); + final String currThreadName = Thread.currentThread().getName(); + try { + Thread.currentThread().setName(StringUtils.format("sql[%s]", stmt.sqlQueryId())); + final ResultSet thePlan = stmt.plan(); + final ByteString results = encodeResults(request.getResultFormat(), thePlan); + return QueryResponse.newBuilder() + .setQueryId(stmt.sqlQueryId()) + .setStatus(QueryStatus.OK) + .setData(results) + .build(); + } + catch (ForbiddenException e) { + return QueryResponse.newBuilder() + .setQueryId(stmt.sqlQueryId()) + .setStatus(QueryStatus.UNAUTHORIZED) + .build(); + } + catch (SqlPlanningException e) { + return QueryResponse.newBuilder() + .setQueryId(stmt.sqlQueryId()) + .setStatus(QueryStatus.INVALID_SQL) + .setErrorMessage(e.getMessage()) + .build(); + } + catch (IOException | RuntimeException e) { + return QueryResponse.newBuilder() + .setQueryId(stmt.sqlQueryId()) + .setStatus(QueryStatus.RUNTIME_ERROR) + .setErrorMessage(e.getMessage()) + .build(); + } + finally { + Thread.currentThread().setName(currThreadName); + } + } + + private ByteString encodeResults(QueryResultFormat queryResultFormat, ResultSet thePlan) throws IOException + { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + org.apache.druid.server.QueryResponse queryResponse = thePlan.run(); + ResultFormat.Writer writer; + switch (queryResultFormat) { + case CSV: + writer = ResultFormat.CSV.createFormatter(out, jsonMapper); + break; + case JSON_ARRAY: + writer = ResultFormat.ARRAY.createFormatter(out, jsonMapper); + break; + case JSON_ARRAY_LINES: + writer = ResultFormat.ARRAYLINES.createFormatter(out, jsonMapper); + break; + case JSON_OBJECT: + throw new UnsupportedOperationException(); // TODO + case JSON_OBJECT_LINES: + throw new UnsupportedOperationException(); // TODO + case PROTOBUF_INLINE: + throw new UnsupportedOperationException(); // TODO + case PROTOBUF_RESPONSE: + throw new UnsupportedOperationException(); // TODO + default: + throw new IAE("Unsupported query result format"); + } + final SqlRowTransformer rowTransformer = thePlan.createRowTransformer(); + return null; + } + + private SqlQueryPlus translateQuery(QueryRequest request) + { + AuthenticationResult authResult = new AuthenticationResult( + "testSuperuser", + AuthConfig.ALLOW_ALL_NAME, + null, null + ); + return SqlQueryPlus.builder() + .sql(request.getQuery()) + .context(translateContext(request)) + .sqlParameters(translateParameters(request)) + .auth(authResult) + .build(); + } + + private Map translateContext(QueryRequest request) + { + ImmutableMap.Builder builder = ImmutableMap.builder(); + if (request.getContextCount() > 0) { + for (Map.Entry entry : request.getContextMap().entrySet()) { + builder.put(entry.getKey(), entry.getValue()); + } + } + return builder.build(); + } + + private List translateParameters(QueryRequest request) + { + if (request.getParametersCount() == 0) { + return null; + } + List params = new ArrayList<>(); + for (QueryParameter value : request.getParametersList()) { + params.add(translateParameter(value)); + } + return params; + } + + private SqlParameter translateParameter(QueryParameter value) + { + switch (value.getValueCase()) { + case ARRAYVALUE: + // Not yet supported: waiting for an open PR + return null; + case DOUBLEVALUE: + return new SqlParameter(SqlType.DOUBLE, value.getDoubleValue()); + case LONGVALUE: + return new SqlParameter(SqlType.BIGINT, value.getLongValue()); + case STRINGVALUE: + return new SqlParameter(SqlType.VARCHAR, value.getStringValue()); + case NULLVALUE: + case VALUE_NOT_SET: + default: + return null; + } + } +} diff --git a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryServer.java b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryServer.java index b2dc3f361761..6ddd1ef409f0 100644 --- a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryServer.java +++ b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryServer.java @@ -1,28 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package org.apache.druid.grpc.server; import io.grpc.Grpc; import io.grpc.InsecureServerCredentials; import io.grpc.Server; import io.grpc.stub.StreamObserver; -import java.io.IOException; -import java.util.concurrent.TimeUnit; -import java.util.logging.Logger; import org.druid.grpc.proto.QueryGrpc; import org.druid.grpc.proto.QueryOuterClass.QueryRequest; import org.druid.grpc.proto.QueryOuterClass.QueryResponse; +import javax.inject.Inject; + +import java.io.IOException; +import java.util.concurrent.TimeUnit; +import java.util.logging.Logger; + public class QueryServer { private static final Logger logger = Logger.getLogger(QueryServer.class.getName()); + private final QueryDriver driver; private Server server; - private void start() throws IOException { + @Inject + public QueryServer(QueryDriver driver) + { + this.driver = driver; + } + + public void start() throws IOException { /* The port on which the server should run */ int port = 50051; server = Grpc.newServerBuilderForPort(port, InsecureServerCredentials.create()) - .addService(new QueryImpl()) + .addService(new QueryImpl(driver)) .build() .start(); logger.info("Server started, listening on " + port); @@ -41,7 +70,7 @@ public void run() { }); } - private void stop() throws InterruptedException { + public void stop() throws InterruptedException { if (server != null) { server.shutdown().awaitTermination(30, TimeUnit.SECONDS); } @@ -50,27 +79,27 @@ private void stop() throws InterruptedException { /** * Await termination on the main thread since the grpc library uses daemon threads. */ - private void blockUntilShutdown() throws InterruptedException { + public void blockUntilShutdown() throws InterruptedException { if (server != null) { server.awaitTermination(); } } - /** - * Main launches the server from the command line. - */ - public static void main(String[] args) throws IOException, InterruptedException { - final QueryServer server = new QueryServer(); - server.start(); - server.blockUntilShutdown(); - } - static class QueryImpl extends QueryGrpc.QueryImplBase { + private final QueryDriver driver; + + public QueryImpl(QueryDriver driver) + { + this.driver = driver; + + } @Override public void submitQuery(QueryRequest request, StreamObserver responseObserver) { + QueryResponse reply = driver.submitQuery(request); + responseObserver.onNext(reply); + responseObserver.onCompleted(); } } - } diff --git a/extensions-contrib/grpc-query/src/main/proto/query.proto b/extensions-contrib/grpc-query/src/main/proto/query.proto index e25cbb79e85c..7fa3b79f46c7 100644 --- a/extensions-contrib/grpc-query/src/main/proto/query.proto +++ b/extensions-contrib/grpc-query/src/main/proto/query.proto @@ -86,4 +86,4 @@ message QueryResponse { optional string errorMessage = 3; repeated ColumnSchema columns = 4; optional bytes data = 5; -} \ No newline at end of file +} diff --git a/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTest.java b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTest.java new file mode 100644 index 000000000000..1443023998aa --- /dev/null +++ b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTest.java @@ -0,0 +1,94 @@ +package org.apache.druid.grpc; + +import io.grpc.CallOptions; +import io.grpc.Grpc; +import io.grpc.InsecureChannelCredentials; +import io.grpc.ManagedChannel; +import io.grpc.StatusRuntimeException; +import org.apache.druid.grpc.server.QueryServer; +import org.druid.grpc.proto.QueryGrpc; +import org.druid.grpc.proto.QueryGrpc.QueryBlockingStub; +import org.druid.grpc.proto.QueryOuterClass.QueryRequest; +import org.druid.grpc.proto.QueryOuterClass.QueryResponse; +import org.druid.grpc.proto.QueryOuterClass.QueryStatus; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.io.IOException; +import java.util.concurrent.TimeUnit; + +import static org.junit.Assert.assertEquals; + +public class GrpcQueryTest +{ + static QueryServer server; + static TestClient client; + + public static class TestClient + { + ManagedChannel channel; + QueryBlockingStub client; + + public TestClient() + { + // Access a service running on the local machine on port 50051 + String target = "localhost:50051"; + // Create a communication channel to the server, known as a Channel. Channels are thread-safe + // and reusable. It is common to create channels at the beginning of your application and reuse + // them until the application shuts down. + // + // For the example we use plaintext insecure credentials to avoid needing TLS certificates. To + // use TLS, use TlsChannelCredentials instead. + channel = Grpc.newChannelBuilder(target, InsecureChannelCredentials.create()) + .build(); + client = QueryGrpc.newBlockingStub(channel); + } + + public void close() throws InterruptedException + { + // ManagedChannels use resources like threads and TCP connections. To prevent leaking these + // resources the channel should be shut down when it will no longer be used. If it may be used + // again leave it running. + channel.shutdownNow().awaitTermination(5, TimeUnit.SECONDS); + } + } + + @BeforeClass + public static void setup() throws IOException + { + server = new QueryServer(); + try { + server.start(); + } + catch (IOException e) { + e.printStackTrace(); + throw e; + } + catch (RuntimeException e) { + e.printStackTrace(); + throw e; + } + client = new TestClient(); + } + + @AfterClass + public static void tearDown() throws InterruptedException + { + if (client != null) { + client.close(); + } + if (server != null) { + server.stop(); + server.blockUntilShutdown(); + } + } + + @Test + public void testBasics() + { + QueryRequest request = QueryRequest.newBuilder().setQuery("SELECT * FROM foo").build(); + QueryResponse response = client.client.submitQuery(request); + assertEquals(QueryStatus.OK, response.getStatus()); + } +} diff --git a/pom.xml b/pom.xml index ac577819c61a..6f6e34123e66 100644 --- a/pom.xml +++ b/pom.xml @@ -215,6 +215,7 @@ extensions-contrib/prometheus-emitter extensions-contrib/opentelemetry-emitter extensions-contrib/kubernetes-overlord-extensions + extensions-contrib/grpc-query distribution diff --git a/server/src/main/java/org/apache/druid/server/QueryResultPusher.java b/server/src/main/java/org/apache/druid/server/QueryResultPusher.java index b25e31d363d8..ce745d5b2104 100644 --- a/server/src/main/java/org/apache/druid/server/QueryResultPusher.java +++ b/server/src/main/java/org/apache/druid/server/QueryResultPusher.java @@ -297,7 +297,7 @@ public interface ResultsWriter extends Closeable /** * Gets the results of running the query. {@link #start} must be called before this method is called. * - * @return the results of running the query as preparted by the {@link #start()} method + * @return the results of running the query as prepared by the {@link #start()} method */ QueryResponse getQueryResponse(); diff --git a/sql/src/main/java/org/apache/druid/sql/SqlPlanningException.java b/sql/src/main/java/org/apache/druid/sql/SqlPlanningException.java index 41dcf6cc02f6..9f027fb270bd 100644 --- a/sql/src/main/java/org/apache/druid/sql/SqlPlanningException.java +++ b/sql/src/main/java/org/apache/druid/sql/SqlPlanningException.java @@ -32,7 +32,6 @@ */ public class SqlPlanningException extends BadQueryException { - public enum PlanningError { SQL_PARSE_ERROR(SQL_PARSE_FAILED_ERROR_CODE, SqlParseException.class.getName()), @@ -103,5 +102,4 @@ private SqlPlanningException( { super(cause, errorCode, errorMessage, errorClass, null); } - } diff --git a/sql/src/main/java/org/apache/druid/sql/http/SqlResource.java b/sql/src/main/java/org/apache/druid/sql/http/SqlResource.java index dad391e6bf04..8d4a13ac969a 100644 --- a/sql/src/main/java/org/apache/druid/sql/http/SqlResource.java +++ b/sql/src/main/java/org/apache/druid/sql/http/SqlResource.java @@ -177,25 +177,21 @@ private static class SqlResourceQueryMetricCounter implements QueryResource.Quer @Override public void incrementSuccess() { - } @Override public void incrementFailed() { - } @Override public void incrementInterrupted() { - } @Override public void incrementTimedOut() { - } } @@ -209,19 +205,16 @@ private SqlResourceQueryResultPusher makePusher(HttpServletRequest req, HttpStat headers.put(SQL_HEADER_RESPONSE_HEADER, SQL_HEADER_VALUE); } - return new SqlResourceQueryResultPusher(req, sqlQueryId, stmt, sqlQuery, headers); + return new SqlResourceQueryResultPusher(req, stmt, sqlQuery, headers); } private class SqlResourceQueryResultPusher extends QueryResultPusher { - - private final String sqlQueryId; private final HttpStatement stmt; private final SqlQuery sqlQuery; public SqlResourceQueryResultPusher( HttpServletRequest req, - String sqlQueryId, HttpStatement stmt, SqlQuery sqlQuery, Map headers @@ -233,11 +226,10 @@ public SqlResourceQueryResultPusher( SqlResource.this.responseContextConfig, SqlResource.this.selfNode, SqlResource.QUERY_METRIC_COUNTER, - sqlQueryId, + stmt.sqlQueryId(), MediaType.APPLICATION_JSON_TYPE, headers ); - this.sqlQueryId = sqlQueryId; this.stmt = stmt; this.sqlQuery = sqlQuery; } @@ -345,9 +337,9 @@ public void recordSuccess(long numBytes) public void recordFailure(Exception e) { if (sqlQuery.queryContext().isDebug()) { - log.warn(e, "Exception while processing sqlQueryId[%s]", sqlQueryId); + log.warn(e, "Exception while processing sqlQueryId[%s]", stmt.sqlQueryId()); } else { - log.noStackTrace().warn(e, "Exception while processing sqlQueryId[%s]", sqlQueryId); + log.noStackTrace().warn(e, "Exception while processing sqlQueryId[%s]", stmt.sqlQueryId()); } stmt.reporter().failed(e); } From d05846ac9914ab8337550943d64bccc60b0ae282 Mon Sep 17 00:00:00 2001 From: Paul Rogers Date: Mon, 16 Jan 2023 17:49:19 -0800 Subject: [PATCH 03/43] Basic functionality works (at least for the Driver) --- extensions-contrib/grpc-query/pom.xml | 7 + .../apache/druid/grpc/server/QueryDriver.java | 347 +++++++++++++++--- .../apache/druid/grpc/server/QueryServer.java | 38 +- .../org/apache/druid/grpc/DriverTest.java | 95 +++++ .../org/apache/druid/grpc/GrpcQueryTest.java | 43 ++- .../druid/grpc/QueryFrameworkFixture.java | 78 ++++ 6 files changed, 542 insertions(+), 66 deletions(-) create mode 100644 extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/DriverTest.java create mode 100644 extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/QueryFrameworkFixture.java diff --git a/extensions-contrib/grpc-query/pom.xml b/extensions-contrib/grpc-query/pom.xml index 4db73fce5bc6..9b2658422604 100644 --- a/extensions-contrib/grpc-query/pom.xml +++ b/extensions-contrib/grpc-query/pom.xml @@ -118,6 +118,13 @@ test-jar test + + org.apache.druid + druid-sql + ${project.parent.version} + test-jar + test + junit junit diff --git a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriver.java b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriver.java index 802590c85ab5..45d8395b4a96 100644 --- a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriver.java +++ b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriver.java @@ -25,21 +25,29 @@ import com.google.inject.Inject; import com.google.protobuf.ByteString; import org.apache.calcite.avatica.SqlType; +import org.apache.calcite.rel.type.RelDataType; import org.apache.druid.guice.annotations.Json; import org.apache.druid.guice.annotations.NativeQuery; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.server.security.AuthConfig; +import org.apache.druid.java.util.common.guava.Accumulator; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.server.security.Access; import org.apache.druid.server.security.AuthenticationResult; import org.apache.druid.server.security.ForbiddenException; import org.apache.druid.sql.DirectStatement; +import org.apache.druid.sql.DirectStatement.ResultSet; import org.apache.druid.sql.SqlPlanningException; import org.apache.druid.sql.SqlQueryPlus; import org.apache.druid.sql.SqlRowTransformer; import org.apache.druid.sql.SqlStatementFactory; -import org.apache.druid.sql.DirectStatement.ResultSet; +import org.apache.druid.sql.calcite.table.RowSignatures; import org.apache.druid.sql.http.ResultFormat; import org.apache.druid.sql.http.SqlParameter; +import org.druid.grpc.proto.QueryOuterClass.ColumnSchema; +import org.druid.grpc.proto.QueryOuterClass.DruidType; import org.druid.grpc.proto.QueryOuterClass.QueryParameter; import org.druid.grpc.proto.QueryOuterClass.QueryRequest; import org.druid.grpc.proto.QueryOuterClass.QueryResponse; @@ -51,7 +59,14 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Optional; +/** + * "Driver" for the gRPC query endpoint. Handles translating the gRPC {@link QueryRequest} + * into Druid's internal formats, running the query, and translating the results into a + * gRPC {@link QueryResponse}. Allows for easier unit testing as we separate the machinery + * of running a query, given the request, from the gRPC server machinery. + */ public class QueryDriver { private final ObjectMapper jsonMapper; @@ -67,25 +82,35 @@ public QueryDriver( this.sqlStatementFactory = Preconditions.checkNotNull(sqlStatementFactory, "sqlStatementFactory"); } - public QueryResponse submitQuery(QueryRequest request) + /** + * First-cut synchronous query handler. Druid prefers to stream results, in + * part to avoid overly-short network timeouts. However, for now, we simply run + * the query within this call and prepare the Protobuf response. Async handling + * can come later. + */ + public QueryResponse submitQuery(QueryRequest request, AuthenticationResult authResult) { - final SqlQueryPlus queryPlus = translateQuery(request); + final SqlQueryPlus queryPlus = translateQuery(request, authResult); final DirectStatement stmt = sqlStatementFactory.directStatement(queryPlus); final String currThreadName = Thread.currentThread().getName(); try { Thread.currentThread().setName(StringUtils.format("sql[%s]", stmt.sqlQueryId())); final ResultSet thePlan = stmt.plan(); - final ByteString results = encodeResults(request.getResultFormat(), thePlan); + final SqlRowTransformer rowTransformer = thePlan.createRowTransformer(); + final ByteString results = encodeResults(request.getResultFormat(), thePlan, rowTransformer); return QueryResponse.newBuilder() .setQueryId(stmt.sqlQueryId()) .setStatus(QueryStatus.OK) .setData(results) + .clearErrorMessage() + .addAllColumns(encodeColumns(rowTransformer)) .build(); } catch (ForbiddenException e) { return QueryResponse.newBuilder() .setQueryId(stmt.sqlQueryId()) .setStatus(QueryStatus.UNAUTHORIZED) + .setErrorMessage(Access.DEFAULT_ERROR_MESSAGE) .build(); } catch (SqlPlanningException e) { @@ -102,48 +127,27 @@ public QueryResponse submitQuery(QueryRequest request) .setErrorMessage(e.getMessage()) .build(); } + // There is a claim that Calcite sometimes throws a java.lang.AssertionError, but we do not have a test that can + // reproduce it checked into the code (the best we have is something that uses mocks to throw an Error, which is + // dubious at best). We keep this just in case, but it might be best to remove it and see where the + // AssertionErrors are coming from and do something to ensure that they don't actually make it out of Calcite + catch (AssertionError e) { + return QueryResponse.newBuilder() + .setQueryId(stmt.sqlQueryId()) + .setStatus(QueryStatus.RUNTIME_ERROR) + .setErrorMessage(e.getMessage()) + .build(); + } finally { Thread.currentThread().setName(currThreadName); } } - private ByteString encodeResults(QueryResultFormat queryResultFormat, ResultSet thePlan) throws IOException + /** + * Convert the rRPC query format to the internal {@link SqlQueryPlus} format. + */ + private SqlQueryPlus translateQuery(QueryRequest request, AuthenticationResult authResult) { - ByteArrayOutputStream out = new ByteArrayOutputStream(); - org.apache.druid.server.QueryResponse queryResponse = thePlan.run(); - ResultFormat.Writer writer; - switch (queryResultFormat) { - case CSV: - writer = ResultFormat.CSV.createFormatter(out, jsonMapper); - break; - case JSON_ARRAY: - writer = ResultFormat.ARRAY.createFormatter(out, jsonMapper); - break; - case JSON_ARRAY_LINES: - writer = ResultFormat.ARRAYLINES.createFormatter(out, jsonMapper); - break; - case JSON_OBJECT: - throw new UnsupportedOperationException(); // TODO - case JSON_OBJECT_LINES: - throw new UnsupportedOperationException(); // TODO - case PROTOBUF_INLINE: - throw new UnsupportedOperationException(); // TODO - case PROTOBUF_RESPONSE: - throw new UnsupportedOperationException(); // TODO - default: - throw new IAE("Unsupported query result format"); - } - final SqlRowTransformer rowTransformer = thePlan.createRowTransformer(); - return null; - } - - private SqlQueryPlus translateQuery(QueryRequest request) - { - AuthenticationResult authResult = new AuthenticationResult( - "testSuperuser", - AuthConfig.ALLOW_ALL_NAME, - null, null - ); return SqlQueryPlus.builder() .sql(request.getQuery()) .context(translateContext(request)) @@ -152,6 +156,14 @@ private SqlQueryPlus translateQuery(QueryRequest request) .build(); } + /** + * Translate the query context from the gRPC format to the internal format. When + * read from REST/JSON, the JSON translator will convert the type of each value + * into a number, Boolean, etc. gRPC has no similar feature. Rather than clutter up + * the gRPC request with typed context values, we rely on the existing code that can + * translate string values to the desired type on the fly. Thus, we build up a + * {@code Map}. + */ private Map translateContext(QueryRequest request) { ImmutableMap.Builder builder = ImmutableMap.builder(); @@ -163,6 +175,10 @@ private Map translateContext(QueryRequest request) return builder.build(); } + /** + * Convert the gRPC parameter format to the internal Druid {@link SqlParameter} + * format. That format is then again translated by the {@link SqlQueryPlus} class. + */ private List translateParameters(QueryRequest request) { if (request.getParametersCount() == 0) { @@ -173,24 +189,247 @@ private List translateParameters(QueryRequest request) params.add(translateParameter(value)); } return params; - } + } private SqlParameter translateParameter(QueryParameter value) { switch (value.getValueCase()) { - case ARRAYVALUE: - // Not yet supported: waiting for an open PR - return null; - case DOUBLEVALUE: - return new SqlParameter(SqlType.DOUBLE, value.getDoubleValue()); - case LONGVALUE: - return new SqlParameter(SqlType.BIGINT, value.getLongValue()); - case STRINGVALUE: - return new SqlParameter(SqlType.VARCHAR, value.getStringValue()); - case NULLVALUE: - case VALUE_NOT_SET: - default: + case ARRAYVALUE: + // Not yet supported: waiting for an open PR + return null; + case DOUBLEVALUE: + return new SqlParameter(SqlType.DOUBLE, value.getDoubleValue()); + case LONGVALUE: + return new SqlParameter(SqlType.BIGINT, value.getLongValue()); + case STRINGVALUE: + return new SqlParameter(SqlType.VARCHAR, value.getStringValue()); + case NULLVALUE: + case VALUE_NOT_SET: + default: + return null; + } + } + + /** + * Translate the column schema from the Druid internal form to the gRPC + * {@link ColumnSchema} form. Note that since the gRPC response returns the + * schema, none of the data formats include a header. This makes the data format + * simpler and cleaner. + */ + private Iterable encodeColumns(SqlRowTransformer rowTransformer) + { + RelDataType rowType = rowTransformer.getRowType(); + final RowSignature signature = RowSignatures.fromRelDataType(rowType.getFieldNames(), rowType); + List cols = new ArrayList<>(); + for (int i = 0; i < rowType.getFieldCount(); i++) { + ColumnSchema col = ColumnSchema.newBuilder() + .setName(signature.getColumnName(i)) + .setSqlType(rowType.getFieldList().get(i).getType().getSqlTypeName().getName()) + .setDruidType(convertDruidType(signature.getColumnType(i))) + .build(); + cols.add(col); + } + return cols; + } + + /** + * Convert from Druid's internal format of the Druid data type to the gRPC form. + */ + private DruidType convertDruidType(Optional colType) + { + if (!colType.isPresent()) { + return DruidType.UNRECOGNIZED; + } + ColumnType druidType = colType.get(); + if (druidType == ColumnType.STRING) { + return DruidType.STRING; + } + if (druidType == ColumnType.STRING_ARRAY) { + return DruidType.STRING_ARRAY; + } + if (druidType == ColumnType.LONG) { + return DruidType.LONG; + } + if (druidType == ColumnType.LONG_ARRAY) { + return DruidType.LONG_ARRAY; + } + if (druidType == ColumnType.FLOAT) { + return DruidType.FLOAT; + } + if (druidType == ColumnType.FLOAT_ARRAY) { + return DruidType.FLOAT_ARRAY; + } + if (druidType == ColumnType.DOUBLE) { + return DruidType.DOUBLE; + } + if (druidType == ColumnType.DOUBLE_ARRAY) { + return DruidType.DOUBLE_ARRAY; + } + if (druidType == ColumnType.UNKNOWN_COMPLEX) { + return DruidType.COMPLEX; + } + return DruidType.UNRECOGNIZED; + } + + /** + * Generic mechanism to write query results to one of the supported gRPC formats. + */ + public interface GrpcResultWriter + { + void start() throws IOException; + void writeRow(Object[] row) throws IOException; + void close() throws IOException; + } + + /** + * Writer for the SQL result formats. Reuses the SQL format writer implementations. + * Note: gRPC does not use the headers: schema information is available in the + * rRPC response. + */ + public static class GrpcResultFormatWriter implements GrpcResultWriter + { + protected final ResultFormat.Writer formatWriter; + protected final SqlRowTransformer rowTransformer; + + public GrpcResultFormatWriter( + final ResultFormat.Writer formatWriter, + final SqlRowTransformer rowTransformer + ) + { + this.formatWriter = formatWriter; + this.rowTransformer = rowTransformer; + } + + @Override + public void start() throws IOException + { + } + + @Override + public void writeRow(Object[] row) throws IOException + { + formatWriter.writeRowStart(); + for (int i = 0; i < rowTransformer.getFieldList().size(); i++) { + final Object value = rowTransformer.transform(row, i); + formatWriter.writeRowField(rowTransformer.getFieldList().get(i), value); + } + formatWriter.writeRowEnd(); + } + + @Override + public void close() throws IOException + { + formatWriter.close(); + } + } + + /** + * Internal runtime exception to pass {@link IOException}s though the + * {@link Sequence} {@link Accumulator} protocol. + */ + private static class ResponseError extends RuntimeException + { + public ResponseError(IOException e) + { + super(e); + } + } + + /** + * Druid query results use a complex {@link Sequence} mechanism. This class uses an + * {@link Accumulator} to walk the results and present each to the associated + * results writer. This is a very rough analogy of the {@code SqlResourceQueryResultPusher} + * in the REST {@code SqlResource} class. + */ + public static class GrpcResultsAccumulator implements Accumulator + { + private final GrpcResultWriter writer; + + public GrpcResultsAccumulator(final GrpcResultWriter writer) + { + this.writer = writer; + } + + public void push(org.apache.druid.server.QueryResponse queryResponse) throws IOException + { + final Sequence results = queryResponse.getResults(); + writer.start(); + try { + results.accumulate(null, this); + } + catch (ResponseError e) { + throw (IOException) e.getCause(); + } + writer.close(); + } + + @Override + public Void accumulate(Void accumulated, Object[] in) + { + try { + writer.writeRow(in); + } + catch (IOException e) { + throw new ResponseError(e); + } return null; } } + + /** + * Convert the query results to a set of bytes to be attached to the query response. + *

+ * This version is pretty basic: the results are materialized as a byte array. That's + * fine for small result sets, but should be rethought for larger result sets. + */ + private ByteString encodeResults( + final QueryResultFormat queryResultFormat, + final ResultSet thePlan, + final SqlRowTransformer rowTransformer + ) throws IOException + { + // Accumulate the results as a byte array. + ByteArrayOutputStream out = new ByteArrayOutputStream(); + GrpcResultWriter writer; + + // For the SQL-supported formats, use the SQL-provided writers. + switch (queryResultFormat) { + case CSV: + writer = new GrpcResultFormatWriter( + ResultFormat.CSV.createFormatter(out, jsonMapper), + rowTransformer + ); + break; + case JSON_ARRAY: + writer = new GrpcResultFormatWriter( + ResultFormat.ARRAY.createFormatter(out, jsonMapper), + rowTransformer + ); + break; + case JSON_ARRAY_LINES: + writer = new GrpcResultFormatWriter( + ResultFormat.ARRAYLINES.createFormatter(out, jsonMapper), + rowTransformer + ); + break; + + // TODO: Provide additional writers for the other formats which we + // want in gRPC. + case JSON_OBJECT: + throw new UnsupportedOperationException(); // TODO + case JSON_OBJECT_LINES: + throw new UnsupportedOperationException(); // TODO + case PROTOBUF_INLINE: + throw new UnsupportedOperationException(); // TODO + + // This is the hard one: encode the results as a Protobuf array. + case PROTOBUF_RESPONSE: + throw new UnsupportedOperationException(); // TODO + default: + throw new IAE("Unsupported query result format"); + } + GrpcResultsAccumulator accumulator = new GrpcResultsAccumulator(writer); + accumulator.push(thePlan.run()); + return ByteString.copyFrom(out.toByteArray()); + } } diff --git a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryServer.java b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryServer.java index 6ddd1ef409f0..08aebaa2eb30 100644 --- a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryServer.java +++ b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryServer.java @@ -23,6 +23,7 @@ import io.grpc.InsecureServerCredentials; import io.grpc.Server; import io.grpc.stub.StreamObserver; +import org.apache.druid.server.security.AuthenticationResult; import org.druid.grpc.proto.QueryGrpc; import org.druid.grpc.proto.QueryOuterClass.QueryRequest; import org.druid.grpc.proto.QueryOuterClass.QueryResponse; @@ -34,6 +35,16 @@ import java.util.logging.Logger; +/** + * Basic gRPC server adapted from the gRPC examples. Delegates to the + * {@link QueryDriver} class to do the actual work of running the query. + *

+ * This class is preliminary. It is good enough for unit tests, but a bit more work + * is needed to integrate this class into the Druid server. One question: must we + * run another web server or is there a way to reuse the existing Jetty server? + *

+ * Also, how will authorization be handled in the gRPC path? + */ public class QueryServer { private static final Logger logger = Logger.getLogger(QueryServer.class.getName()); @@ -47,7 +58,8 @@ public QueryServer(QueryDriver driver) this.driver = driver; } - public void start() throws IOException { + public void start() throws IOException + { /* The port on which the server should run */ int port = 50051; server = Grpc.newServerBuilderForPort(port, InsecureServerCredentials.create()) @@ -55,14 +67,17 @@ public void start() throws IOException { .build() .start(); logger.info("Server started, listening on " + port); - Runtime.getRuntime().addShutdownHook(new Thread() { + Runtime.getRuntime().addShutdownHook(new Thread() + { @Override - public void run() { + public void run() + { // Use stderr here since the logger may have been reset by its JVM shutdown hook. System.err.println("*** shutting down gRPC server since JVM is shutting down"); try { QueryServer.this.stop(); - } catch (InterruptedException e) { + } + catch (InterruptedException e) { e.printStackTrace(System.err); } System.err.println("*** server shut down"); @@ -70,7 +85,8 @@ public void run() { }); } - public void stop() throws InterruptedException { + public void stop() throws InterruptedException + { if (server != null) { server.shutdown().awaitTermination(30, TimeUnit.SECONDS); } @@ -79,14 +95,15 @@ public void stop() throws InterruptedException { /** * Await termination on the main thread since the grpc library uses daemon threads. */ - public void blockUntilShutdown() throws InterruptedException { + public void blockUntilShutdown() throws InterruptedException + { if (server != null) { server.awaitTermination(); } } - static class QueryImpl extends QueryGrpc.QueryImplBase { - + static class QueryImpl extends QueryGrpc.QueryImplBase + { private final QueryDriver driver; public QueryImpl(QueryDriver driver) @@ -94,10 +111,13 @@ public QueryImpl(QueryDriver driver) this.driver = driver; } + @Override public void submitQuery(QueryRequest request, StreamObserver responseObserver) { - QueryResponse reply = driver.submitQuery(request); + // TODO: How will we get the auth result for gRPC? + AuthenticationResult authResult = null; + QueryResponse reply = driver.submitQuery(request, authResult); responseObserver.onNext(reply); responseObserver.onCompleted(); } diff --git a/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/DriverTest.java b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/DriverTest.java new file mode 100644 index 000000000000..6e3a6efc5a1b --- /dev/null +++ b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/DriverTest.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.grpc; + +import org.apache.druid.grpc.server.QueryDriver; +import org.apache.druid.sql.calcite.util.CalciteTests; +import org.druid.grpc.proto.QueryOuterClass.ColumnSchema; +import org.druid.grpc.proto.QueryOuterClass.DruidType; +import org.druid.grpc.proto.QueryOuterClass.QueryRequest; +import org.druid.grpc.proto.QueryOuterClass.QueryResponse; +import org.druid.grpc.proto.QueryOuterClass.QueryResultFormat; +import org.druid.grpc.proto.QueryOuterClass.QueryStatus; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class DriverTest +{ + @ClassRule + public static TemporaryFolder temporaryFolder = new TemporaryFolder(); + private static QueryFrameworkFixture frameworkFixture; + private static QueryDriver driver; + + @BeforeClass + public static void setup() throws IOException + { + frameworkFixture = new QueryFrameworkFixture(temporaryFolder.newFolder()); + driver = new QueryDriver( + frameworkFixture.jsonMapper(), + frameworkFixture.statementFactory() + ); + } + + @Test + public void testBasics() + { + String sql = "SELECT __time, dim2 FROM foo"; + QueryRequest request = QueryRequest.newBuilder() + .setQuery(sql) + .setResultFormat(QueryResultFormat.CSV) + .build(); + QueryResponse response = driver.submitQuery(request, CalciteTests.REGULAR_USER_AUTH_RESULT); + + assertEquals(QueryStatus.OK, response.getStatus()); + assertFalse(response.hasErrorMessage()); + assertTrue(response.getQueryId().length() > 5); + List columns = response.getColumnsList(); + assertEquals(2, columns.size()); + ColumnSchema col = columns.get(0); + assertEquals("__time", col.getName()); + assertEquals("TIMESTAMP", col.getSqlType()); + assertEquals(DruidType.LONG, col.getDruidType()); + col = columns.get(1); + assertEquals("dim2", col.getName()); + assertEquals("VARCHAR", col.getSqlType()); + assertEquals(DruidType.STRING, col.getDruidType()); + + List expectedResults = Arrays.asList( + "2000-01-01T00:00:00.000Z,a", + "2000-01-02T00:00:00.000Z,", + "2000-01-03T00:00:00.000Z,", + "2001-01-01T00:00:00.000Z,a", + "2001-01-02T00:00:00.000Z,abc", + "2001-01-03T00:00:00.000Z," + ); + String results = response.getData().toStringUtf8(); + assertEquals(expectedResults, Arrays.asList(results.split("\n"))); + } +} diff --git a/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTest.java b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTest.java index 1443023998aa..74133179de54 100644 --- a/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTest.java +++ b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTest.java @@ -1,10 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package org.apache.druid.grpc; -import io.grpc.CallOptions; import io.grpc.Grpc; import io.grpc.InsecureChannelCredentials; import io.grpc.ManagedChannel; -import io.grpc.StatusRuntimeException; +import org.apache.druid.grpc.server.QueryDriver; import org.apache.druid.grpc.server.QueryServer; import org.druid.grpc.proto.QueryGrpc; import org.druid.grpc.proto.QueryGrpc.QueryBlockingStub; @@ -13,7 +31,9 @@ import org.druid.grpc.proto.QueryOuterClass.QueryStatus; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; +import org.junit.rules.TemporaryFolder; import java.io.IOException; import java.util.concurrent.TimeUnit; @@ -22,6 +42,9 @@ public class GrpcQueryTest { + @ClassRule + public static TemporaryFolder temporaryFolder = new TemporaryFolder(); + private static QueryFrameworkFixture frameworkFixture; static QueryServer server; static TestClient client; @@ -57,7 +80,12 @@ public void close() throws InterruptedException @BeforeClass public static void setup() throws IOException { - server = new QueryServer(); + frameworkFixture = new QueryFrameworkFixture(temporaryFolder.newFolder()); + QueryDriver driver = new QueryDriver( + frameworkFixture.jsonMapper(), + frameworkFixture.statementFactory() + ); + server = new QueryServer(driver); try { server.start(); } @@ -84,6 +112,15 @@ public static void tearDown() throws InterruptedException } } + /** + * Do a very basic query. + *

+ * Note: at present this will fail with a cryptic internal error because the version + * of Guava which Druid uses is not compatible with the version that gRPC requires. + * We'll need to do some shading. + *

+ * java.lang.NoSuchMethodError: 'void com.google.common.base.Preconditions.checkArgument(boolean, java.lang.String, char, java.lang.Object)' + */ @Test public void testBasics() { diff --git a/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/QueryFrameworkFixture.java b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/QueryFrameworkFixture.java new file mode 100644 index 000000000000..da946e33756d --- /dev/null +++ b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/QueryFrameworkFixture.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.grpc; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.query.topn.TopNQueryConfig; +import org.apache.druid.server.security.AuthConfig; +import org.apache.druid.sql.SqlStatementFactory; +import org.apache.druid.sql.calcite.BaseCalciteQueryTest; +import org.apache.druid.sql.calcite.util.SqlTestFramework; +import org.apache.druid.sql.calcite.util.SqlTestFramework.PlannerComponentSupplier; +import org.apache.druid.sql.calcite.util.SqlTestFramework.PlannerFixture; +import org.apache.druid.sql.calcite.util.SqlTestFramework.QueryComponentSupplier; +import org.apache.druid.sql.calcite.util.SqlTestFramework.StandardComponentSupplier; +import org.apache.druid.sql.calcite.util.SqlTestFramework.StandardPlannerComponentSupplier; + +import java.io.File; + +/** + * Test-specific "fixture" to package up the "Calcite test" query infrastructure + * into a form needed to run gRPC unit tests. gRPC requires that the statement factory + * be available at class setup time. Normally, the Calcite tests build this object + * per test. + *

+ * This fixture uses default values, which should be fine for gRPC tests: we want to + * test the gRPC mechanism, not the planner or query stack. If any customization is + * needed, it can be done here with a custom component supplier, etc. + */ +public class QueryFrameworkFixture +{ + private final SqlTestFramework queryFramework; + private final PlannerFixture plannerFixture; + + public QueryFrameworkFixture(final File temporaryFolder) + { + QueryComponentSupplier componentSupplier = new StandardComponentSupplier( + temporaryFolder + ); + SqlTestFramework.Builder builder = new SqlTestFramework.Builder(componentSupplier) + .minTopNThreshold(TopNQueryConfig.DEFAULT_MIN_TOPN_THRESHOLD) + .mergeBufferCount(0); + queryFramework = builder.build(); + PlannerComponentSupplier plannerComponentSupplier = new StandardPlannerComponentSupplier(); + AuthConfig authConfig = new AuthConfig(); + plannerFixture = queryFramework.plannerFixture( + plannerComponentSupplier, + BaseCalciteQueryTest.PLANNER_CONFIG_DEFAULT, + authConfig + ); + } + + public SqlStatementFactory statementFactory() + { + return plannerFixture.statementFactory(); + } + + public ObjectMapper jsonMapper() + { + return queryFramework.queryJsonMapper(); + } +} From 05aac53ee7aed0a92a12b9f4d61dad4c661c9a2f Mon Sep 17 00:00:00 2001 From: Paul Rogers Date: Tue, 17 Jan 2023 09:00:36 -0800 Subject: [PATCH 04/43] Fix protobuf enums Report stats & close statement --- .../apache/druid/grpc/server/QueryDriver.java | 36 ++++++++++++-- .../grpc-query/src/main/proto/query.proto | 48 +++++++++++-------- 2 files changed, 61 insertions(+), 23 deletions(-) diff --git a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriver.java b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriver.java index 45d8395b4a96..673631bcf8c0 100644 --- a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriver.java +++ b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriver.java @@ -28,7 +28,6 @@ import org.apache.calcite.rel.type.RelDataType; import org.apache.druid.guice.annotations.Json; import org.apache.druid.guice.annotations.NativeQuery; -import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.guava.Accumulator; import org.apache.druid.java.util.common.guava.Sequence; @@ -69,6 +68,17 @@ */ public class QueryDriver { + /** + * Internal runtime exception to report request errors. + */ + private static class RequestError extends RuntimeException + { + public RequestError(String msg) + { + super(msg); + } + } + private final ObjectMapper jsonMapper; private final SqlStatementFactory sqlStatementFactory; @@ -98,6 +108,8 @@ public QueryResponse submitQuery(QueryRequest request, AuthenticationResult auth final ResultSet thePlan = stmt.plan(); final SqlRowTransformer rowTransformer = thePlan.createRowTransformer(); final ByteString results = encodeResults(request.getResultFormat(), thePlan, rowTransformer); + stmt.reporter().succeeded(0); // TODO: real byte count (of payload) + stmt.close(); return QueryResponse.newBuilder() .setQueryId(stmt.sqlQueryId()) .setStatus(QueryStatus.OK) @@ -107,13 +119,26 @@ public QueryResponse submitQuery(QueryRequest request, AuthenticationResult auth .build(); } catch (ForbiddenException e) { + stmt.reporter().failed(e); + stmt.close(); return QueryResponse.newBuilder() .setQueryId(stmt.sqlQueryId()) .setStatus(QueryStatus.UNAUTHORIZED) .setErrorMessage(Access.DEFAULT_ERROR_MESSAGE) .build(); } + catch (RequestError e) { + stmt.reporter().failed(e); + stmt.close(); + return QueryResponse.newBuilder() + .setQueryId(stmt.sqlQueryId()) + .setStatus(QueryStatus.REQUEST_ERROR) + .setErrorMessage(Access.DEFAULT_ERROR_MESSAGE) + .build(); + } catch (SqlPlanningException e) { + stmt.reporter().failed(e); + stmt.close(); return QueryResponse.newBuilder() .setQueryId(stmt.sqlQueryId()) .setStatus(QueryStatus.INVALID_SQL) @@ -121,6 +146,8 @@ public QueryResponse submitQuery(QueryRequest request, AuthenticationResult auth .build(); } catch (IOException | RuntimeException e) { + stmt.reporter().failed(e); + stmt.close(); return QueryResponse.newBuilder() .setQueryId(stmt.sqlQueryId()) .setStatus(QueryStatus.RUNTIME_ERROR) @@ -132,6 +159,8 @@ public QueryResponse submitQuery(QueryRequest request, AuthenticationResult auth // dubious at best). We keep this just in case, but it might be best to remove it and see where the // AssertionErrors are coming from and do something to ensure that they don't actually make it out of Calcite catch (AssertionError e) { + stmt.reporter().failed(e); + stmt.close(); return QueryResponse.newBuilder() .setQueryId(stmt.sqlQueryId()) .setStatus(QueryStatus.RUNTIME_ERROR) @@ -205,8 +234,9 @@ private SqlParameter translateParameter(QueryParameter value) return new SqlParameter(SqlType.VARCHAR, value.getStringValue()); case NULLVALUE: case VALUE_NOT_SET: - default: return null; + default: + throw new RequestError("Invalid parameter type: " + value.getValueCase().name()); } } @@ -426,7 +456,7 @@ private ByteString encodeResults( case PROTOBUF_RESPONSE: throw new UnsupportedOperationException(); // TODO default: - throw new IAE("Unsupported query result format"); + throw new RequestError("Unsupported query result format"); } GrpcResultsAccumulator accumulator = new GrpcResultsAccumulator(writer); accumulator.push(thePlan.run()); diff --git a/extensions-contrib/grpc-query/src/main/proto/query.proto b/extensions-contrib/grpc-query/src/main/proto/query.proto index 7fa3b79f46c7..4131797146a9 100644 --- a/extensions-contrib/grpc-query/src/main/proto/query.proto +++ b/extensions-contrib/grpc-query/src/main/proto/query.proto @@ -24,13 +24,15 @@ service Query { } enum QueryResultFormat { - CSV = 0; - JSON_OBJECT = 1; - JSON_ARRAY = 2; - JSON_OBJECT_LINES = 3; - JSON_ARRAY_LINES = 4; - PROTOBUF_INLINE = 5; - PROTOBUF_RESPONSE = 6; + UNKNOWN_FORMAT = 0; // Default value. An old server will see this value + // (and fail the request) if a new client passes a new format. + CSV = 1; + JSON_OBJECT = 2; + JSON_ARRAY = 3; + JSON_OBJECT_LINES = 4; + JSON_ARRAY_LINES = 5; + PROTOBUF_INLINE = 6; + PROTOBUF_RESPONSE = 7; } message QueryParameter { @@ -56,22 +58,28 @@ message QueryRequest { } enum QueryStatus { - OK = 0; - UNAUTHORIZED = 1; - INVALID_SQL = 2; - RUNTIME_ERROR = 3; + UNKNOWN_STATUS = 0; // Default value which means "unknown failure". Older clients + // will see this value if a future version adds a new failure + // type. + OK = 1; + UNAUTHORIZED = 2; + REQUEST_ERROR = 3; + INVALID_SQL = 4; + RUNTIME_ERROR = 5; } enum DruidType { - STRING = 0; - LONG = 1; - DOUBLE = 2; - FLOAT = 3; - STRING_ARRAY = 4; - LONG_ARRAY = 5; - DOUBLE_ARRAY = 6; - FLOAT_ARRAY = 7; - COMPLEX = 8; + UNKNOWN_TYPE = 0; // Default value for unknown, or for future new values + // as seen by old clients. + STRING = 1; + LONG = 2; + DOUBLE = 3; + FLOAT = 4; + STRING_ARRAY = 5; + LONG_ARRAY = 6; + DOUBLE_ARRAY = 7; + FLOAT_ARRAY = 8; + COMPLEX = 9; } message ColumnSchema { From 0b217af35e10463be987fd9acbe033e227413345 Mon Sep 17 00:00:00 2001 From: Paul Rogers Date: Sun, 22 Jan 2023 17:39:34 -0800 Subject: [PATCH 05/43] Snapshot --- extensions-contrib/grpc-query/README.md | 103 +++++++ extensions-contrib/grpc-query/pom.xml | 289 ++++++++---------- .../apache/druid/grpc/server/QueryDriver.java | 14 +- .../apache/druid/grpc/server/QueryServer.java | 6 +- .../org/apache/druid/grpc/DriverTest.java | 12 +- .../org/apache/druid/grpc/GrpcQueryTest.java | 18 +- extensions-contrib/grpc-shaded/pom.xml | 147 +++++++++ .../src/main/proto/query.proto | 2 +- pom.xml | 1 + 9 files changed, 408 insertions(+), 184 deletions(-) create mode 100644 extensions-contrib/grpc-query/README.md create mode 100644 extensions-contrib/grpc-shaded/pom.xml rename extensions-contrib/{grpc-query => grpc-shaded}/src/main/proto/query.proto (97%) diff --git a/extensions-contrib/grpc-query/README.md b/extensions-contrib/grpc-query/README.md new file mode 100644 index 000000000000..99ea787da1f8 --- /dev/null +++ b/extensions-contrib/grpc-query/README.md @@ -0,0 +1,103 @@ +# gRPC Query Extension for Druid + +This extension provides a gRPC API for SQL queries. + +Druid uses REST as its RPC protocol. Druid has a large variety of REST operations +including query, ingest jobs, monitoring, configuration and many more. Although +REST is a universally supported RPC format, is not the only one in use. This +extension allows gRPC-based clients to issue SQL queries. + +Druid is optimized for high-concurrency, low-complexity queries that return a +small result set (a few thousand rows at most). The small-query focus allows +Druid to offer a simple, stateless request/response REST API. This gRPC API +follows that Druid pattern: it is optimized for simple queries and follows +Druid's request/response model. APIs such as JDBC can handle larger results +because they stateful: a client can request pages of results using multiple +API calls. This API does not support paging: the entire result set is returned +in the response, resulting in a API which is fast for small queries, and not +suitable for larger result sets. + +## Use Cases + +The gRPC query extension can be used in two ways, depending on the selected +result format. + +### CSV or JSON Response Format + +The simplest way to use the gRPC extension is to send a query request that +uses CSV or JSON as the return format. The client simply pulls the results +from the response and does something useful with them. For the CSV format, +headers can be created from the column metadata in the response message. + +### Protobuf Response Format + +Some applications want to use Protobuf as the result format. In this case, +the extension encodes Protobuf-encoded rows as the binary payload of the query +response. This works for an application which uses a fixed set of queries, each +of which is carefully designed to power one application, say a dashboard. The +(simplified) message flow is: + +```text ++-----------+ query -> +-------+ +| Dashboard | -- gRPC --> | Druid | ++-----------+ <- data +-------+ +``` + +In practice, there may be multiple proxy layers: one on the application side, and +the Router on the Druid side. + +The dashboard displays a fixed set of reports and charts. Each of those sends a +well-defined query specified as part of the application. The returned data is thus +both well-known and fixed for each query. The set of queries is fixed by the contents +of the dashboard. That is, this is not an ad-hoc query use case. + +Because the queries are locked down, and are part of the application, the set of valid +result sets is also well known and locked down. Given this well-controlled use case, it +is possible to use a pre-defined Protobuf message to represent the results of each distinct +query. (Protobuf is a compiled format: the solution works only because the set of messages +are well known. It would not work for the ad-hoc case in which each query has a different +result set schema.) + +To be very clear: the application has a fixed set of queries to be sent to Druid via gRPC. +For each query, there is a fixed Protobuf response format defined by the application. +No other queries, aside from this well-known set, will be sent to the gRPC endpoint using +the Protobuf response format. If the set of queries is not well-defined, use the CSV +or JSON response format instead. + +## Installation + +## Usage + +See the `src/main/proto/query.proto` file in the `grpc-shade` project for the request and +response message formats. The request message format closely follows the REST JSON message +format. The response is optimized for gRPC: it contains an error (if the request fails), +or the result schema and result data as a binary payload. You can query the gRPC endpoint +with any gRPC client. + +## Implementation Notes + +The extension is made up of two projects. Druid uses a different version of Guava than +does rRPC. To work around this, the `grpc-shade` project creates a shaded jar that includes +both gRPC and Guava. Since the rRPC compiler generates references to Guava, it turns out we +must generate Java from the `.proto` files _before_ shading, so the `.proto` files, and +the generated Java code, also reside in the `grpc-shade` project. One unfortunate aspect +of shading is that your IDE will not be able to find the source attachments for the rRPC +classes. The result is very tedious debugging. A workaround is to manually attach the +original gRPC source jars for each gRPC source file. + +This project contains several components: + +* Guice modules and associated server initialization code. +* Netty-based gRPC server. +* A "driver" that performs the actual query and generates the results. + +## Acknowledgements + +This is not the first project to have created a gRPC API for Druid. Others include: + +* [[Proposal] define a RPC protocol for querying data, support apache Arrow as data + exchange interface](https://github.com/apache/druid/issues/3891) +* [gRPC Druid extension PoC](https://github.com/ndolgov/gruid) +* [Druid gRPC-json server extension](https://github.com/apache/druid/pull/6798) + +Full credit goes to those who have gone this way before. diff --git a/extensions-contrib/grpc-query/pom.xml b/extensions-contrib/grpc-query/pom.xml index 9b2658422604..07dcaf160c98 100644 --- a/extensions-contrib/grpc-query/pom.xml +++ b/extensions-contrib/grpc-query/pom.xml @@ -19,11 +19,11 @@ --> - 4.0.0 - org.apache.druid.extensions.contrib - grpc-query - grpc-query - grpc-query + 4.0.0 + org.apache.druid.extensions.contrib + grpc-query + grpc-query + grpc-query org.apache.druid @@ -32,157 +32,136 @@ ../../pom.xml - - - org.apache.druid - druid-core - ${project.parent.version} - provided - - - org.apache.druid - druid-server - ${project.parent.version} - provided - - - org.apache.druid - druid-processing - ${project.parent.version} - provided - - - org.apache.druid - druid-sql - ${project.parent.version} - provided - - - io.grpc - grpc-netty-shaded - 1.52.0 - runtime - - - io.grpc - grpc-protobuf - 1.52.0 - - - io.grpc - grpc-stub - 1.52.0 - + + 1.52.0 + 3.2.1 + - - com.fasterxml.jackson.module - jackson-module-guice - provided - - - com.google.inject - guice - provided - - - com.fasterxml.jackson.core - jackson-databind - provided - - - com.fasterxml.jackson.core - jackson-core - provided - - - com.google.inject.extensions - guice-multibindings - provided - - - com.google.guava - guava - provided - - - com.google.code.findbugs - jsr305 - provided - + + + org.apache.druid + druid-core + ${project.parent.version} + provided + + + org.apache.druid + druid-server + ${project.parent.version} + provided + + + org.apache.druid + druid-processing + ${project.parent.version} + provided + + + org.apache.druid + druid-sql + ${project.parent.version} + provided + + + org.apache.druid.extensions.contrib + druid-shaded-grpc + ${project.parent.version} + - - - org.apache.druid - druid-core - ${project.parent.version} - test-jar - test - - - org.apache.druid - druid-sql - ${project.parent.version} - test-jar - test - - - junit - junit - test - - + + com.fasterxml.jackson.module + jackson-module-guice + provided + + + com.google.inject + guice + provided + + + com.fasterxml.jackson.core + jackson-databind + provided + + + com.fasterxml.jackson.core + jackson-core + provided + + + com.google.inject.extensions + guice-multibindings + provided + + + com.google.guava + guava + provided + + + com.google.code.findbugs + jsr305 + provided + + + io.netty + netty-codec-http2 + provided + + + io.netty + netty + provided + - - - - kr.motd.maven - os-maven-plugin - 1.6.2 - - - - - org.xolstice.maven.plugins - protobuf-maven-plugin - 0.6.1 - - com.google.protobuf:protoc:3.21.7:exe:${os.detected.classifier} - grpc-java - io.grpc:protoc-gen-grpc-java:1.52.0:exe:${os.detected.classifier} - - - - - compile - compile-custom - - - - - - org.apache.maven.plugins - maven-assembly-plugin - - false - - jar-with-dependencies - - - - true - - - - - - make-assembly - package - - single - - - - - - + + + org.apache.druid + druid-core + ${project.parent.version} + test-jar + test + + + org.apache.druid + druid-sql + ${project.parent.version} + test-jar + test + + + junit + junit + test + + + + + + + org.apache.maven.plugins + maven-assembly-plugin + + false + + jar-with-dependencies + + + + true + + + + + + make-assembly + package + + single + + + + + + diff --git a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriver.java b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriver.java index 673631bcf8c0..2e0d69607865 100644 --- a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriver.java +++ b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriver.java @@ -45,13 +45,13 @@ import org.apache.druid.sql.calcite.table.RowSignatures; import org.apache.druid.sql.http.ResultFormat; import org.apache.druid.sql.http.SqlParameter; -import org.druid.grpc.proto.QueryOuterClass.ColumnSchema; -import org.druid.grpc.proto.QueryOuterClass.DruidType; -import org.druid.grpc.proto.QueryOuterClass.QueryParameter; -import org.druid.grpc.proto.QueryOuterClass.QueryRequest; -import org.druid.grpc.proto.QueryOuterClass.QueryResponse; -import org.druid.grpc.proto.QueryOuterClass.QueryResultFormat; -import org.druid.grpc.proto.QueryOuterClass.QueryStatus; +import org.apache.druid.grpc.proto.QueryOuterClass.ColumnSchema; +import org.apache.druid.grpc.proto.QueryOuterClass.DruidType; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryParameter; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryRequest; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryResponse; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryResultFormat; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryStatus; import java.io.ByteArrayOutputStream; import java.io.IOException; diff --git a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryServer.java b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryServer.java index 08aebaa2eb30..29540e7eb618 100644 --- a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryServer.java +++ b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryServer.java @@ -24,9 +24,9 @@ import io.grpc.Server; import io.grpc.stub.StreamObserver; import org.apache.druid.server.security.AuthenticationResult; -import org.druid.grpc.proto.QueryGrpc; -import org.druid.grpc.proto.QueryOuterClass.QueryRequest; -import org.druid.grpc.proto.QueryOuterClass.QueryResponse; +import org.apache.druid.grpc.proto.QueryGrpc; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryRequest; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryResponse; import javax.inject.Inject; diff --git a/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/DriverTest.java b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/DriverTest.java index 6e3a6efc5a1b..e85c99b62b9b 100644 --- a/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/DriverTest.java +++ b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/DriverTest.java @@ -21,12 +21,12 @@ import org.apache.druid.grpc.server.QueryDriver; import org.apache.druid.sql.calcite.util.CalciteTests; -import org.druid.grpc.proto.QueryOuterClass.ColumnSchema; -import org.druid.grpc.proto.QueryOuterClass.DruidType; -import org.druid.grpc.proto.QueryOuterClass.QueryRequest; -import org.druid.grpc.proto.QueryOuterClass.QueryResponse; -import org.druid.grpc.proto.QueryOuterClass.QueryResultFormat; -import org.druid.grpc.proto.QueryOuterClass.QueryStatus; +import org.apache.druid.grpc.proto.QueryOuterClass.ColumnSchema; +import org.apache.druid.grpc.proto.QueryOuterClass.DruidType; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryRequest; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryResponse; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryResultFormat; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryStatus; import org.junit.BeforeClass; import org.junit.ClassRule; import org.junit.Test; diff --git a/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTest.java b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTest.java index 74133179de54..77ac4d0630cc 100644 --- a/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTest.java +++ b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTest.java @@ -24,11 +24,11 @@ import io.grpc.ManagedChannel; import org.apache.druid.grpc.server.QueryDriver; import org.apache.druid.grpc.server.QueryServer; -import org.druid.grpc.proto.QueryGrpc; -import org.druid.grpc.proto.QueryGrpc.QueryBlockingStub; -import org.druid.grpc.proto.QueryOuterClass.QueryRequest; -import org.druid.grpc.proto.QueryOuterClass.QueryResponse; -import org.druid.grpc.proto.QueryOuterClass.QueryStatus; +import org.apache.druid.grpc.proto.QueryGrpc; +import org.apache.druid.grpc.proto.QueryGrpc.QueryBlockingStub; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryRequest; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryResponse; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryStatus; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.ClassRule; @@ -55,7 +55,7 @@ public static class TestClient public TestClient() { - // Access a service running on the local machine on port 50051 + // Access a service running on the local machine on port 50051 String target = "localhost:50051"; // Create a communication channel to the server, known as a Channel. Channels are thread-safe // and reusable. It is common to create channels at the beginning of your application and reuse @@ -114,12 +114,6 @@ public static void tearDown() throws InterruptedException /** * Do a very basic query. - *

- * Note: at present this will fail with a cryptic internal error because the version - * of Guava which Druid uses is not compatible with the version that gRPC requires. - * We'll need to do some shading. - *

- * java.lang.NoSuchMethodError: 'void com.google.common.base.Preconditions.checkArgument(boolean, java.lang.String, char, java.lang.Object)' */ @Test public void testBasics() diff --git a/extensions-contrib/grpc-shaded/pom.xml b/extensions-contrib/grpc-shaded/pom.xml new file mode 100644 index 000000000000..496f372497f4 --- /dev/null +++ b/extensions-contrib/grpc-shaded/pom.xml @@ -0,0 +1,147 @@ + + + + + + 4.0.0 + org.apache.druid.extensions.contrib + druid-shaded-grpc + druid-shaded-grpc + Shaded gRPC jar + + + org.apache.druid + druid + 26.0.0-SNAPSHOT + ../../pom.xml + + + + 1.52.0 + 3.2.1 + + 20.0 + + + + + io.grpc + grpc-core + ${grpc.version} + + + io.grpc + grpc-netty + ${grpc.version} + + + io.grpc + grpc-protobuf + ${grpc.version} + + + io.grpc + grpc-stub + ${grpc.version} + + + com.google.guava + guava + ${guava.version} + + + + + + + kr.motd.maven + os-maven-plugin + 1.6.2 + + + + + org.xolstice.maven.plugins + protobuf-maven-plugin + 0.6.1 + + com.google.protobuf:protoc:3.21.7:exe:${os.detected.classifier} + grpc-java + io.grpc:protoc-gen-grpc-java:1.52.0:exe:${os.detected.classifier} + + + + + compile + compile-custom + + + + + + org.apache.maven.plugins + maven-shade-plugin + ${maven.shade.plugin.version} + + + package + + shade + + + + + + com.google.guava + org.apache.druid.grpc.shaded.com.google.guava + + + com.google.common.base + org.apache.druid.grpc.shaded.com.google.common.base + + + com.google.common.util.concurrent + org.apache.druid.grpc.shaded.com.google.common.util.concurrent + + + ${project.build.directory}/dependency-reduced-pom.xml + + + + + com.google.code.gson:gson + com.google.android:annotations + org.codehaus.mojo:animal-sniffer-annotations + com.google.errorprone:error_prone_annotations + io.perfmark:perfmark-api + io.netty:* + com.google.code.findbugs + + + + + + + + + diff --git a/extensions-contrib/grpc-query/src/main/proto/query.proto b/extensions-contrib/grpc-shaded/src/main/proto/query.proto similarity index 97% rename from extensions-contrib/grpc-query/src/main/proto/query.proto rename to extensions-contrib/grpc-shaded/src/main/proto/query.proto index 4131797146a9..6ea3e8e6d062 100644 --- a/extensions-contrib/grpc-query/src/main/proto/query.proto +++ b/extensions-contrib/grpc-shaded/src/main/proto/query.proto @@ -15,7 +15,7 @@ syntax = "proto3"; -option java_package = "org.druid.grpc.proto"; +option java_package = "org.apache.druid.grpc.proto"; package druidGrpc; diff --git a/pom.xml b/pom.xml index 6f6e34123e66..45189974bbf2 100644 --- a/pom.xml +++ b/pom.xml @@ -215,6 +215,7 @@ extensions-contrib/prometheus-emitter extensions-contrib/opentelemetry-emitter extensions-contrib/kubernetes-overlord-extensions + extensions-contrib/grpc-shaded extensions-contrib/grpc-query distribution From 24cde0cb02cea37e54a74c36a0f70caf6fbc9c1f Mon Sep 17 00:00:00 2001 From: Paul Rogers Date: Mon, 23 Jan 2023 18:04:42 -0800 Subject: [PATCH 06/43] Shaded jar Simple server + query test works --- extensions-contrib/grpc-query/README.md | 22 +++++ .../apache/druid/grpc/server/QueryDriver.java | 34 ++++--- .../apache/druid/grpc/server/QueryServer.java | 10 +- .../org/apache/druid/grpc/DriverTest.java | 4 +- .../druid/grpc/GrpcQueryClientTest.java | 96 +++++++++++++++++++ .../org/apache/druid/grpc/GrpcQueryTest.java | 10 +- extensions-contrib/grpc-shaded/pom.xml | 8 +- .../grpc-shaded/src/main/proto/query.proto | 30 +++++- 8 files changed, 191 insertions(+), 23 deletions(-) create mode 100644 extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryClientTest.java diff --git a/extensions-contrib/grpc-query/README.md b/extensions-contrib/grpc-query/README.md index 99ea787da1f8..f9ba9e5f5ed4 100644 --- a/extensions-contrib/grpc-query/README.md +++ b/extensions-contrib/grpc-query/README.md @@ -91,6 +91,28 @@ This project contains several components: * Netty-based gRPC server. * A "driver" that performs the actual query and generates the results. +## Debugging + +Debugging of the gRPC stack is difficult since the shaded jar loses source attachments. + +Logging helps. gRPC logging is not enabled via Druid's logging system. Intead, [create +the following `logging.properties` file](https://stackoverflow.com/questions/50243717/grpc-logger-level): + +```text +handlers=java.util.logging.ConsoleHandler +io.grpc.level=FINE +java.util.logging.ConsoleHandler.level=FINE +java.util.logging.ConsoleHandler.formatter=java.util.logging.SimpleFormatter +``` + +Then, pass the following on the command line: + +```text +-Djava.util.logging.config.file=logging.properties +``` + +Adjust the path to the file depending on where you put the file. + ## Acknowledgements This is not the first project to have created a gRPC API for Druid. Others include: diff --git a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriver.java b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriver.java index 2e0d69607865..454c202b568f 100644 --- a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriver.java +++ b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriver.java @@ -26,6 +26,13 @@ import com.google.protobuf.ByteString; import org.apache.calcite.avatica.SqlType; import org.apache.calcite.rel.type.RelDataType; +import org.apache.druid.grpc.proto.QueryOuterClass.ColumnSchema; +import org.apache.druid.grpc.proto.QueryOuterClass.DruidType; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryParameter; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryRequest; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryResponse; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryResultFormat; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryStatus; import org.apache.druid.guice.annotations.Json; import org.apache.druid.guice.annotations.NativeQuery; import org.apache.druid.java.util.common.StringUtils; @@ -45,13 +52,6 @@ import org.apache.druid.sql.calcite.table.RowSignatures; import org.apache.druid.sql.http.ResultFormat; import org.apache.druid.sql.http.SqlParameter; -import org.apache.druid.grpc.proto.QueryOuterClass.ColumnSchema; -import org.apache.druid.grpc.proto.QueryOuterClass.DruidType; -import org.apache.druid.grpc.proto.QueryOuterClass.QueryParameter; -import org.apache.druid.grpc.proto.QueryOuterClass.QueryRequest; -import org.apache.druid.grpc.proto.QueryOuterClass.QueryResponse; -import org.apache.druid.grpc.proto.QueryOuterClass.QueryResultFormat; -import org.apache.druid.grpc.proto.QueryOuterClass.QueryStatus; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -100,11 +100,21 @@ public QueryDriver( */ public QueryResponse submitQuery(QueryRequest request, AuthenticationResult authResult) { - final SqlQueryPlus queryPlus = translateQuery(request, authResult); + final SqlQueryPlus queryPlus; + try { + queryPlus = translateQuery(request, authResult); + } + catch (RuntimeException e) { + return QueryResponse.newBuilder() + .setQueryId("") + .setStatus(QueryStatus.REQUEST_ERROR) + .setErrorMessage(e.getMessage()) + .build(); + } final DirectStatement stmt = sqlStatementFactory.directStatement(queryPlus); final String currThreadName = Thread.currentThread().getName(); try { - Thread.currentThread().setName(StringUtils.format("sql[%s]", stmt.sqlQueryId())); + Thread.currentThread().setName(StringUtils.format("grpc-sql[%s]", stmt.sqlQueryId())); final ResultSet thePlan = stmt.plan(); final SqlRowTransformer rowTransformer = thePlan.createRowTransformer(); final ByteString results = encodeResults(request.getResultFormat(), thePlan, rowTransformer); @@ -133,7 +143,7 @@ public QueryResponse submitQuery(QueryRequest request, AuthenticationResult auth return QueryResponse.newBuilder() .setQueryId(stmt.sqlQueryId()) .setStatus(QueryStatus.REQUEST_ERROR) - .setErrorMessage(Access.DEFAULT_ERROR_MESSAGE) + .setErrorMessage(e.getMessage()) .build(); } catch (SqlPlanningException e) { @@ -268,7 +278,7 @@ private Iterable encodeColumns(SqlRowTransformer rowTran private DruidType convertDruidType(Optional colType) { if (!colType.isPresent()) { - return DruidType.UNRECOGNIZED; + return DruidType.UNKNOWN_TYPE; } ColumnType druidType = colType.get(); if (druidType == ColumnType.STRING) { @@ -298,7 +308,7 @@ private DruidType convertDruidType(Optional colType) if (druidType == ColumnType.UNKNOWN_COMPLEX) { return DruidType.COMPLEX; } - return DruidType.UNRECOGNIZED; + return DruidType.UNKNOWN_TYPE; } /** diff --git a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryServer.java b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryServer.java index 29540e7eb618..036411b87ab0 100644 --- a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryServer.java +++ b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryServer.java @@ -23,10 +23,11 @@ import io.grpc.InsecureServerCredentials; import io.grpc.Server; import io.grpc.stub.StreamObserver; -import org.apache.druid.server.security.AuthenticationResult; import org.apache.druid.grpc.proto.QueryGrpc; import org.apache.druid.grpc.proto.QueryOuterClass.QueryRequest; import org.apache.druid.grpc.proto.QueryOuterClass.QueryResponse; +import org.apache.druid.server.security.AuthConfig; +import org.apache.druid.server.security.AuthenticationResult; import javax.inject.Inject; @@ -109,14 +110,17 @@ static class QueryImpl extends QueryGrpc.QueryImplBase public QueryImpl(QueryDriver driver) { this.driver = driver; - } @Override public void submitQuery(QueryRequest request, StreamObserver responseObserver) { // TODO: How will we get the auth result for gRPC? - AuthenticationResult authResult = null; + AuthenticationResult authResult = new AuthenticationResult( + "superUser", + AuthConfig.ALLOW_ALL_NAME, + null, null + ); QueryResponse reply = driver.submitQuery(request, authResult); responseObserver.onNext(reply); responseObserver.onCompleted(); diff --git a/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/DriverTest.java b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/DriverTest.java index e85c99b62b9b..98fa27acd617 100644 --- a/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/DriverTest.java +++ b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/DriverTest.java @@ -19,14 +19,14 @@ package org.apache.druid.grpc; -import org.apache.druid.grpc.server.QueryDriver; -import org.apache.druid.sql.calcite.util.CalciteTests; import org.apache.druid.grpc.proto.QueryOuterClass.ColumnSchema; import org.apache.druid.grpc.proto.QueryOuterClass.DruidType; import org.apache.druid.grpc.proto.QueryOuterClass.QueryRequest; import org.apache.druid.grpc.proto.QueryOuterClass.QueryResponse; import org.apache.druid.grpc.proto.QueryOuterClass.QueryResultFormat; import org.apache.druid.grpc.proto.QueryOuterClass.QueryStatus; +import org.apache.druid.grpc.server.QueryDriver; +import org.apache.druid.sql.calcite.util.CalciteTests; import org.junit.BeforeClass; import org.junit.ClassRule; import org.junit.Test; diff --git a/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryClientTest.java b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryClientTest.java new file mode 100644 index 000000000000..4a5119f9f321 --- /dev/null +++ b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryClientTest.java @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.grpc; + +import io.grpc.Grpc; +import io.grpc.InsecureChannelCredentials; +import io.grpc.ManagedChannel; +import org.apache.druid.grpc.proto.QueryGrpc; +import org.apache.druid.grpc.proto.QueryGrpc.QueryBlockingStub; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryRequest; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryResponse; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryStatus; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.io.IOException; +import java.util.concurrent.TimeUnit; + +import static org.junit.Assert.assertEquals; + +public class GrpcQueryClientTest +{ + static TestClient client; + + public static class TestClient + { + ManagedChannel channel; + QueryBlockingStub client; + + public TestClient() + { + // Access a service running on the local machine on port 50051 + String target = "localhost:50051"; + // Create a communication channel to the server, known as a Channel. Channels are thread-safe + // and reusable. It is common to create channels at the beginning of your application and reuse + // them until the application shuts down. + // + // For the example we use plaintext insecure credentials to avoid needing TLS certificates. To + // use TLS, use TlsChannelCredentials instead. + channel = Grpc.newChannelBuilder(target, InsecureChannelCredentials.create()) + .build(); + client = QueryGrpc.newBlockingStub(channel); + } + + public void close() throws InterruptedException + { + // ManagedChannels use resources like threads and TCP connections. To prevent leaking these + // resources the channel should be shut down when it will no longer be used. If it may be used + // again leave it running. + channel.shutdownNow().awaitTermination(5, TimeUnit.SECONDS); + } + } + + @BeforeClass + public static void setup() throws IOException + { + client = new TestClient(); + } + + @AfterClass + public static void tearDown() throws InterruptedException + { + if (client != null) { + client.close(); + } + } + + /** + * Do a very basic query. + */ + @Test + public void testBasics() + { + QueryRequest request = QueryRequest.newBuilder().setQuery("SELECT * FROM foo").build(); + QueryResponse response = client.client.submitQuery(request); + assertEquals(QueryStatus.OK, response.getStatus()); + } +} diff --git a/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTest.java b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTest.java index 77ac4d0630cc..d5b784c27a94 100644 --- a/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTest.java +++ b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTest.java @@ -22,13 +22,14 @@ import io.grpc.Grpc; import io.grpc.InsecureChannelCredentials; import io.grpc.ManagedChannel; -import org.apache.druid.grpc.server.QueryDriver; -import org.apache.druid.grpc.server.QueryServer; import org.apache.druid.grpc.proto.QueryGrpc; import org.apache.druid.grpc.proto.QueryGrpc.QueryBlockingStub; import org.apache.druid.grpc.proto.QueryOuterClass.QueryRequest; import org.apache.druid.grpc.proto.QueryOuterClass.QueryResponse; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryResultFormat; import org.apache.druid.grpc.proto.QueryOuterClass.QueryStatus; +import org.apache.druid.grpc.server.QueryDriver; +import org.apache.druid.grpc.server.QueryServer; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.ClassRule; @@ -118,7 +119,10 @@ public static void tearDown() throws InterruptedException @Test public void testBasics() { - QueryRequest request = QueryRequest.newBuilder().setQuery("SELECT * FROM foo").build(); + QueryRequest request = QueryRequest.newBuilder() + .setQuery("SELECT * FROM foo") + .setResultFormat(QueryResultFormat.CSV) + .build(); QueryResponse response = client.client.submitQuery(request); assertEquals(QueryStatus.OK, response.getStatus()); } diff --git a/extensions-contrib/grpc-shaded/pom.xml b/extensions-contrib/grpc-shaded/pom.xml index 496f372497f4..eda254f1dda3 100644 --- a/extensions-contrib/grpc-shaded/pom.xml +++ b/extensions-contrib/grpc-shaded/pom.xml @@ -60,7 +60,12 @@ io.grpc grpc-stub ${grpc.version} - + + + io.grpc + grpc-context + ${grpc.version} + com.google.guava guava @@ -133,7 +138,6 @@ com.google.android:annotations org.codehaus.mojo:animal-sniffer-annotations com.google.errorprone:error_prone_annotations - io.perfmark:perfmark-api io.netty:* com.google.code.findbugs diff --git a/extensions-contrib/grpc-shaded/src/main/proto/query.proto b/extensions-contrib/grpc-shaded/src/main/proto/query.proto index 6ea3e8e6d062..41ac0788d61e 100644 --- a/extensions-contrib/grpc-shaded/src/main/proto/query.proto +++ b/extensions-contrib/grpc-shaded/src/main/proto/query.proto @@ -23,6 +23,8 @@ service Query { rpc SubmitQuery (QueryRequest) returns (QueryResponse) {} } +// Defines the format of the query results. Must be one of the +// non-unknown formats. enum QueryResultFormat { UNKNOWN_FORMAT = 0; // Default value. An old server will see this value // (and fail the request) if a new client passes a new format. @@ -31,10 +33,16 @@ enum QueryResultFormat { JSON_ARRAY = 3; JSON_OBJECT_LINES = 4; JSON_ARRAY_LINES = 5; + + // The protobuf formats also require that the protobufMessageName be set in + // the query request. PROTOBUF_INLINE = 6; PROTOBUF_RESPONSE = 7; } +// Value for a query parameter. The value is essentially a variant of the +// supported parameter types. The type chosen here must match (or be converable +// to) the type of the corresponding expression in the SQL statement. message QueryParameter { oneof value { bool nullValue = 1; @@ -45,6 +53,9 @@ message QueryParameter { } } +// Query pararameter value for string array properties. At present, string +// arrays are primarily used by the MSQ engine, which is not yet available +// via the gRPC API. message StringArray { repeated string value = 1; } @@ -53,8 +64,13 @@ message QueryRequest { string query = 1; QueryResultFormat resultFormat = 2; map context = 3; + // Query parameters. If your query is SELECT * FROM foo WHERE x = ? AND y > ? + // Then you would include two parameters in the order in which the question + // marks lexically appear in the query. repeated QueryParameter parameters = 4; - optional string responseMessage = 5; + // The name of the Protobuf message to encode the response if the + // resultFormat is one of the PROTOBUF formats. + optional string protobufMessageName = 5; } enum QueryStatus { @@ -91,7 +107,19 @@ message ColumnSchema { message QueryResponse { string queryId = 1; QueryStatus status = 2; + + // Error message if the query fails. Not set if the query succeeds. optional string errorMessage = 3; + + // The schema of the returned results. This schema is redundant for the + // JSON and Protobuf formats. It can be used to generate column heads, + // and understand types, for the CSV result format. repeated ColumnSchema columns = 4; + + // The query response, encoded using the requested response format. + // Note that the entire response is placed into a single messages. As + // a result, this RPC is intended ONLY for queries that return small + // result sets. It will perform poorly (and consume excess memory) if + // used for large result sets. optional bytes data = 5; } From d404f216fe319aafc8b4d81df6ea28f8e3fa9086 Mon Sep 17 00:00:00 2001 From: Paul Rogers Date: Wed, 25 Jan 2023 13:38:05 -0800 Subject: [PATCH 07/43] Snaphot: one attempt to resolve dependencies --- distribution/pom.xml | 2 + extensions-contrib/grpc-query/README.md | 82 +++++++++- extensions-contrib/grpc-query/pom.xml | 148 +++++++++--------- .../druid/grpc/guice/GrpcQueryModule.java | 10 +- .../grpc/server/GrpcEndpointInitializer.java | 93 +++++++++++ .../druid/grpc/server/GrpcQueryConfig.java | 50 ++++++ .../apache/druid/grpc/server/QueryDriver.java | 8 +- .../apache/druid/grpc/server/QueryServer.java | 66 ++------ .../druid/grpc/server/QueryService.java | 56 +++++++ ...rg.apache.druid.initialization.DruidModule | 2 +- .../org/apache/druid/grpc/GrpcQueryTest.java | 2 +- ...ientTest.java => GrpcQueryTestClient.java} | 64 ++++---- .../org/apache/druid/grpc/TestServer.java | 61 ++++++++ extensions-contrib/grpc-shaded/pom.xml | 34 ++-- 14 files changed, 498 insertions(+), 180 deletions(-) create mode 100644 extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/GrpcEndpointInitializer.java create mode 100644 extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/GrpcQueryConfig.java create mode 100644 extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryService.java rename extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/{GrpcQueryClientTest.java => GrpcQueryTestClient.java} (65%) create mode 100644 extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/TestServer.java diff --git a/distribution/pom.xml b/distribution/pom.xml index 7b92e3436b45..c547ec278422 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -633,6 +633,8 @@ org.apache.druid.extensions.contrib:aliyun-oss-extensions -c org.apache.druid.extensions.contrib:opentelemetry-emitter + -c + org.apache.druid.extensions.contrib:grpc-query diff --git a/extensions-contrib/grpc-query/README.md b/extensions-contrib/grpc-query/README.md index f9ba9e5f5ed4..4916f81ead50 100644 --- a/extensions-contrib/grpc-query/README.md +++ b/extensions-contrib/grpc-query/README.md @@ -1,3 +1,22 @@ + + # gRPC Query Extension for Druid This extension provides a gRPC API for SQL queries. @@ -57,7 +76,7 @@ is possible to use a pre-defined Protobuf message to represent the results of ea query. (Protobuf is a compiled format: the solution works only because the set of messages are well known. It would not work for the ad-hoc case in which each query has a different result set schema.) - +--- To be very clear: the application has a fixed set of queries to be sent to Druid via gRPC. For each query, there is a fixed Protobuf response format defined by the application. No other queries, aside from this well-known set, will be sent to the gRPC endpoint using @@ -66,9 +85,36 @@ or JSON response format instead. ## Installation +The gRPC query extension is a "contrib" extension and is not installed by default when +you install Druid. Instead, you must install it manually. + +In development, you can build Druid with all the "contrib" extensions. When building +Druid, include the `-P bundle-contrib-exts` in place of the `-P dist` option. + +In production, follow the [Druid documentation](https://druid.apache.org/docs/latest/development/extensions.html). +Use the `pull-deps` command to install the `org.apache.druid.extensions.contrib:grpc-query` +extension. + +To enable the extension, add the following to the load list in +`_commmon/common.runtime.properties`: + +```text +druid.extensions.loadList=[..., "grpc-query"] +``` + +Adding the extension to the load list automatically enables the extension. + +Then, enable and configure the extension in `broker/runtime.properties`: + +```text +druid.grpcQuery.port=50051 +``` + +The default port is 50051 (preliminary). + ## Usage -See the `src/main/proto/query.proto` file in the `grpc-shade` project for the request and +See the `src/main/proto/query.proto` file in the `grpc-shaded` project for the request and response message formats. The request message format closely follows the REST JSON message format. The response is optimized for gRPC: it contains an error (if the request fails), or the result schema and result data as a binary payload. You can query the gRPC endpoint @@ -93,6 +139,38 @@ This project contains several components: ## Debugging +Debugging of the gRPC extension requires extra care. + +### Debugging in an IDE + +To debug the gRPC extension in a production-like environment, you'll want to debug the +extension in a running Broker. The easiest way to do this is: + +* Build your branch. Use the `-P bundle-contrib-exts` flag in place of `-P dist`, as described + above. +* Create an install from the distribution produced above. +* Use the `single-server/micro-quickstart` config for debugging. +* Configure the installation using the steps above. +* Modify the Supervisor config for your config to comment out the line that launches + the broker. Use the hash (`#`) character to comment out the line. +* In your IDE, define a launch configuration for the Broker. + * The launch command is `server broker` + * Add the following JVM arguments: + +```text +--add-exports java.base/jdk.internal.perf=ALL-UNNAMED +--add-exports jdk.management/com.sun.management.internal=ALL-UNNAMED +``` + + * Define `grpc-query` as a project dependency. (This is for Eclipse; IntelliJ may differ.) + * Configure the class path to include the common and Broker properties files. + * Explicitly add the `extension/grpc-query/druid-shaded-grpc--SNAPSHOT.jar` in + the class path. +* Launch the micro-quickstart cluster. +* Launch the Broker in your IDE. + +### gRPC Logging + Debugging of the gRPC stack is difficult since the shaded jar loses source attachments. Logging helps. gRPC logging is not enabled via Druid's logging system. Intead, [create diff --git a/extensions-contrib/grpc-query/pom.xml b/extensions-contrib/grpc-query/pom.xml index 07dcaf160c98..8df9fcf1bd20 100644 --- a/extensions-contrib/grpc-query/pom.xml +++ b/extensions-contrib/grpc-query/pom.xml @@ -57,110 +57,118 @@ provided - org.apache.druid - druid-sql - ${project.parent.version} - provided - - - org.apache.druid.extensions.contrib - druid-shaded-grpc - ${project.parent.version} + org.apache.druid + druid-sql + ${project.parent.version} + provided - - com.fasterxml.jackson.module - jackson-module-guice - provided + org.apache.druid.extensions.contrib + druid-shaded-grpc + ${project.parent.version} - com.google.inject - guice - provided + com.fasterxml.jackson.module + jackson-module-guice + provided - com.fasterxml.jackson.core - jackson-databind - provided + com.google.inject + guice + provided - com.fasterxml.jackson.core - jackson-core - provided + com.fasterxml.jackson.core + jackson-databind + provided - com.google.inject.extensions - guice-multibindings - provided + com.fasterxml.jackson.core + jackson-core + provided - com.google.guava - guava - provided + com.google.inject.extensions + guice-multibindings + provided - com.google.code.findbugs - jsr305 - provided + com.google.guava + guava + provided - io.netty - netty-codec-http2 + com.google.code.findbugs + jsr305 provided io.netty - netty - provided + netty-codec-http2 + ${netty4.version} - org.apache.druid - druid-core - ${project.parent.version} - test-jar - test + org.apache.druid + druid-core + ${project.parent.version} + test-jar + test + + + org.apache.druid + druid-sql + ${project.parent.version} + test-jar + test - org.apache.druid - druid-sql - ${project.parent.version} - test-jar - test + junit + junit + test - junit - junit - test + org.apache.druid + druid-server + ${project.parent.version} + test-jar + test + + + org.apache.druid + druid-processing + ${project.parent.version} + test-jar + test - org.apache.maven.plugins - maven-assembly-plugin - - false - - jar-with-dependencies - - - - true - - - - - - make-assembly - package - - single - - - + org.apache.maven.plugins + maven-assembly-plugin + + false + + jar-with-dependencies + + + + true + + + + + + make-assembly + package + + single + + + diff --git a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/guice/GrpcQueryModule.java b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/guice/GrpcQueryModule.java index d5cae6eb86cd..6621a92ed95c 100644 --- a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/guice/GrpcQueryModule.java +++ b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/guice/GrpcQueryModule.java @@ -20,13 +20,21 @@ package org.apache.druid.grpc.guice; import com.google.inject.Binder; +import org.apache.druid.discovery.NodeRole; +import org.apache.druid.grpc.server.GrpcEndpointInitializer; +import org.apache.druid.grpc.server.GrpcQueryConfig; +import org.apache.druid.guice.JsonConfigProvider; +import org.apache.druid.guice.LifecycleModule; +import org.apache.druid.guice.annotations.LoadScope; import org.apache.druid.initialization.DruidModule; +@LoadScope(roles = NodeRole.BROKER_JSON_NAME) public class GrpcQueryModule implements DruidModule { @Override public void configure(Binder binder) { - // TODO Auto-generated method stub + JsonConfigProvider.bind(binder, GrpcQueryConfig.CONFIG_BASE, GrpcQueryConfig.class); + LifecycleModule.register(binder, GrpcEndpointInitializer.class); } } diff --git a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/GrpcEndpointInitializer.java b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/GrpcEndpointInitializer.java new file mode 100644 index 000000000000..8a577f431819 --- /dev/null +++ b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/GrpcEndpointInitializer.java @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.grpc.server; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.guice.ManageLifecycle; +import org.apache.druid.guice.annotations.Json; +import org.apache.druid.guice.annotations.NativeQuery; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.lifecycle.LifecycleStart; +import org.apache.druid.java.util.common.lifecycle.LifecycleStop; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.sql.SqlStatementFactory; + +import javax.inject.Inject; + +import java.io.IOException; + +@ManageLifecycle +public class GrpcEndpointInitializer +{ + private static final Logger log = new Logger(GrpcEndpointInitializer.class); + + private final GrpcQueryConfig config; + private final QueryDriver driver; + + private QueryServer server; + + @Inject + public GrpcEndpointInitializer( + GrpcQueryConfig config, + final @Json ObjectMapper jsonMapper, + final @NativeQuery SqlStatementFactory sqlStatementFactory + ) + { + this.config = config; + this.driver = new QueryDriver(jsonMapper, sqlStatementFactory); + } + + @LifecycleStart + public void start() + { +// String foo = io.netty.handler.codec.http2.Http2Headers.PseudoHeaderName.METHOD.name(); +// final ClassLoader oldLoader = Thread.currentThread().getContextClassLoader(); +// +// try { +// Thread.currentThread().setContextClassLoader(GrpcEndpointInitializer.class.getClassLoader()); + + server = new QueryServer(config.getPort(), driver); + try { + server.start(); + } catch (IOException e) { + log.error(e, "Fatal error: gRPC query server startup failed"); + throw new ISE(e, "Fatal error: grpc query server startup failed"); + } catch (Throwable t) { + log.error(t, "Fatal error: gRPC query server startup failed"); + throw t; + } +// } finally { +// Thread.currentThread().setContextClassLoader(oldLoader); +// } + } + + @LifecycleStop + public void stop() + { + if (server != null) { + try { + server.blockUntilShutdown(); + } catch (InterruptedException e) { + log.warn(e, "gRPC query server shutdown failed"); + } + server = null; + } + } +} diff --git a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/GrpcQueryConfig.java b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/GrpcQueryConfig.java new file mode 100644 index 000000000000..5b8361cf56f6 --- /dev/null +++ b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/GrpcQueryConfig.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.grpc.server; + +import com.fasterxml.jackson.annotation.JsonProperty; + +import javax.validation.constraints.Max; + +public class GrpcQueryConfig +{ + public static final String CONFIG_BASE = "druid.grpcQuery"; + + @JsonProperty + @Max(0xffff) + private int port = 50051; + + public GrpcQueryConfig() + { + } + + public GrpcQueryConfig(int port) + { + this.port = port; + } + + /** + * @return the port to accept gRPC client connections on + */ + public int getPort() + { + return port; + } +} diff --git a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriver.java b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriver.java index 454c202b568f..74591edf1645 100644 --- a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriver.java +++ b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriver.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; -import com.google.inject.Inject; import com.google.protobuf.ByteString; import org.apache.calcite.avatica.SqlType; import org.apache.calcite.rel.type.RelDataType; @@ -33,8 +32,6 @@ import org.apache.druid.grpc.proto.QueryOuterClass.QueryResponse; import org.apache.druid.grpc.proto.QueryOuterClass.QueryResultFormat; import org.apache.druid.grpc.proto.QueryOuterClass.QueryStatus; -import org.apache.druid.guice.annotations.Json; -import org.apache.druid.guice.annotations.NativeQuery; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.guava.Accumulator; import org.apache.druid.java.util.common.guava.Sequence; @@ -82,10 +79,9 @@ public RequestError(String msg) private final ObjectMapper jsonMapper; private final SqlStatementFactory sqlStatementFactory; - @Inject public QueryDriver( - final @Json ObjectMapper jsonMapper, - final @NativeQuery SqlStatementFactory sqlStatementFactory + final ObjectMapper jsonMapper, + final SqlStatementFactory sqlStatementFactory ) { this.jsonMapper = Preconditions.checkNotNull(jsonMapper, "jsonMapper"); diff --git a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryServer.java b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryServer.java index 036411b87ab0..7470fac90c75 100644 --- a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryServer.java +++ b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryServer.java @@ -22,18 +22,11 @@ import io.grpc.Grpc; import io.grpc.InsecureServerCredentials; import io.grpc.Server; -import io.grpc.stub.StreamObserver; -import org.apache.druid.grpc.proto.QueryGrpc; -import org.apache.druid.grpc.proto.QueryOuterClass.QueryRequest; -import org.apache.druid.grpc.proto.QueryOuterClass.QueryResponse; -import org.apache.druid.server.security.AuthConfig; -import org.apache.druid.server.security.AuthenticationResult; +import org.apache.druid.java.util.common.logger.Logger; import javax.inject.Inject; - import java.io.IOException; import java.util.concurrent.TimeUnit; -import java.util.logging.Logger; /** @@ -41,49 +34,32 @@ * {@link QueryDriver} class to do the actual work of running the query. *

* This class is preliminary. It is good enough for unit tests, but a bit more work - * is needed to integrate this class into the Druid server. One question: must we - * run another web server or is there a way to reuse the existing Jetty server? + * is needed to integrate this class into the Druid server. *

* Also, how will authorization be handled in the gRPC path? */ public class QueryServer { - private static final Logger logger = Logger.getLogger(QueryServer.class.getName()); + private static final Logger log = new Logger(QueryServer.class); + private final int port; private final QueryDriver driver; private Server server; @Inject - public QueryServer(QueryDriver driver) + public QueryServer(int port, QueryDriver driver) { + this.port = port; this.driver = driver; } public void start() throws IOException { - /* The port on which the server should run */ - int port = 50051; - server = Grpc.newServerBuilderForPort(port, InsecureServerCredentials.create()) - .addService(new QueryImpl(driver)) + server = Grpc.newServerBuilderForPort(port, InsecureServerCredentials.create()) + .addService(new QueryService(driver)) .build() .start(); - logger.info("Server started, listening on " + port); - Runtime.getRuntime().addShutdownHook(new Thread() - { - @Override - public void run() - { - // Use stderr here since the logger may have been reset by its JVM shutdown hook. - System.err.println("*** shutting down gRPC server since JVM is shutting down"); - try { - QueryServer.this.stop(); - } - catch (InterruptedException e) { - e.printStackTrace(System.err); - } - System.err.println("*** server shut down"); - } - }); + log.info("Server started, listening on " + port); } public void stop() throws InterruptedException @@ -102,28 +78,4 @@ public void blockUntilShutdown() throws InterruptedException server.awaitTermination(); } } - - static class QueryImpl extends QueryGrpc.QueryImplBase - { - private final QueryDriver driver; - - public QueryImpl(QueryDriver driver) - { - this.driver = driver; - } - - @Override - public void submitQuery(QueryRequest request, StreamObserver responseObserver) - { - // TODO: How will we get the auth result for gRPC? - AuthenticationResult authResult = new AuthenticationResult( - "superUser", - AuthConfig.ALLOW_ALL_NAME, - null, null - ); - QueryResponse reply = driver.submitQuery(request, authResult); - responseObserver.onNext(reply); - responseObserver.onCompleted(); - } - } } diff --git a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryService.java b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryService.java new file mode 100644 index 000000000000..54ae92d8d3f5 --- /dev/null +++ b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryService.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.grpc.server; + +import io.grpc.stub.StreamObserver; +import org.apache.druid.grpc.proto.QueryGrpc; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryRequest; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryResponse; +import org.apache.druid.server.security.AuthConfig; +import org.apache.druid.server.security.AuthenticationResult; + +/** + * Implementation of the gRPC Query service. Provides a single method + * to run a query using the "driver" that holds the actual Druid SQL + * logic. + */ +class QueryService extends QueryGrpc.QueryImplBase +{ + private final QueryDriver driver; + + public QueryService(QueryDriver driver) + { + this.driver = driver; + } + + @Override + public void submitQuery(QueryRequest request, StreamObserver responseObserver) + { + // TODO: How will we get the auth result for gRPC? + AuthenticationResult authResult = new AuthenticationResult( + "superUser", + AuthConfig.ALLOW_ALL_NAME, + null, null + ); + QueryResponse reply = driver.submitQuery(request, authResult); + responseObserver.onNext(reply); + responseObserver.onCompleted(); + } +} diff --git a/extensions-contrib/grpc-query/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule b/extensions-contrib/grpc-query/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule index d1e9a4fc482a..e35e9a026993 100644 --- a/extensions-contrib/grpc-query/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule +++ b/extensions-contrib/grpc-query/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule @@ -13,4 +13,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -org.apache.druid.grpc.GrpcQueryModule +org.apache.druid.grpc.guice.GrpcQueryModule diff --git a/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTest.java b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTest.java index d5b784c27a94..7ef0ac292690 100644 --- a/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTest.java +++ b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTest.java @@ -86,7 +86,7 @@ public static void setup() throws IOException frameworkFixture.jsonMapper(), frameworkFixture.statementFactory() ); - server = new QueryServer(driver); + server = new QueryServer(50051, driver); try { server.start(); } diff --git a/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryClientTest.java b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTestClient.java similarity index 65% rename from extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryClientTest.java rename to extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTestClient.java index 4a5119f9f321..70e838039445 100644 --- a/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryClientTest.java +++ b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTestClient.java @@ -24,22 +24,23 @@ import io.grpc.ManagedChannel; import org.apache.druid.grpc.proto.QueryGrpc; import org.apache.druid.grpc.proto.QueryGrpc.QueryBlockingStub; +import org.apache.druid.grpc.proto.QueryOuterClass.ColumnSchema; import org.apache.druid.grpc.proto.QueryOuterClass.QueryRequest; import org.apache.druid.grpc.proto.QueryOuterClass.QueryResponse; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryResultFormat; import org.apache.druid.grpc.proto.QueryOuterClass.QueryStatus; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Test; +import org.apache.druid.java.util.common.StringUtils; -import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.concurrent.TimeUnit; -import static org.junit.Assert.assertEquals; - -public class GrpcQueryClientTest +/** + * Super-simple client which makes a single query request and prints + * the response. Useful because Druid provides no other rRPC client + * to use to test the rRPC endpoint. + */ +public class GrpcQueryTestClient { - static TestClient client; - public static class TestClient { ManagedChannel channel; @@ -69,28 +70,33 @@ public void close() throws InterruptedException } } - @BeforeClass - public static void setup() throws IOException + public static void main(String[] args) { - client = new TestClient(); - } - - @AfterClass - public static void tearDown() throws InterruptedException - { - if (client != null) { - client.close(); + if (args.length != 1) { + System.err.println("Usage: sql-query"); + System.exit(1); } - } - - /** - * Do a very basic query. - */ - @Test - public void testBasics() - { - QueryRequest request = QueryRequest.newBuilder().setQuery("SELECT * FROM foo").build(); + TestClient client = new TestClient(); + QueryRequest request = QueryRequest.newBuilder() + .setQuery(args[0]) + .setResultFormat(QueryResultFormat.CSV) + .build(); QueryResponse response = client.client.submitQuery(request); - assertEquals(QueryStatus.OK, response.getStatus()); + if (response.getStatus() != QueryStatus.OK) { + System.err.println("Failed: " + response.getStatus().name()); + System.err.println(response.getErrorMessage()); + System.exit(1); + } + System.out.println("Columns:"); + for (ColumnSchema col : response.getColumnsList()) { + System.out.println(StringUtils.format("%s %s (%s)", col.getName(), col.getSqlType(), col.getDruidType().name())); + } + System.out.println("Data:"); + System.out.println(response.getData().toString(StandardCharsets.UTF_8)); + try { + client.close(); + } catch (InterruptedException e) { + // Ignore; + } } } diff --git a/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/TestServer.java b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/TestServer.java new file mode 100644 index 000000000000..324101a30670 --- /dev/null +++ b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/TestServer.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.grpc; + +import org.apache.druid.grpc.server.GrpcEndpointInitializer; +import org.apache.druid.grpc.server.GrpcQueryConfig; + +import java.io.File; + +/** + * Super-simple test server that uses the CalciteTests setup. + */ +public class TestServer +{ + private static QueryFrameworkFixture frameworkFixture; + private GrpcEndpointInitializer serverInit; + + public static void main(String[] args) + { + new TestServer().run(); + } + + public void run() + { + frameworkFixture = new QueryFrameworkFixture(new File("/tmp/druid")); + GrpcQueryConfig config = new GrpcQueryConfig(50051); + serverInit = new GrpcEndpointInitializer( + config, + frameworkFixture.jsonMapper(), + frameworkFixture.statementFactory() + ); + serverInit.start(); + Runtime.getRuntime().addShutdownHook(new Thread() + { + @Override + public void run() + { + // Use stderr here since the logger may have been reset by its JVM shutdown hook. + System.err.println("*** shutting down gRPC server since JVM is shutting down"); + serverInit.stop(); + } + }); + } +} diff --git a/extensions-contrib/grpc-shaded/pom.xml b/extensions-contrib/grpc-shaded/pom.xml index eda254f1dda3..8a2bb0497eb1 100644 --- a/extensions-contrib/grpc-shaded/pom.xml +++ b/extensions-contrib/grpc-shaded/pom.xml @@ -40,31 +40,39 @@ 20.0 + + + + io.grpc + grpc-bom + ${grpc.version} + pom + import + + + + io.grpc - grpc-core - ${grpc.version} - - - io.grpc - grpc-netty - ${grpc.version} + grpc-api io.grpc grpc-protobuf - ${grpc.version} io.grpc - grpc-stub - ${grpc.version} + grpc-stub + + + com.google.protobuf + protobuf-java + ${protobuf.version} io.grpc - grpc-context - ${grpc.version} + grpc-netty com.google.guava @@ -130,7 +138,7 @@ From 7efcdc8807b563b592e36bfd6412885de83a78d5 Mon Sep 17 00:00:00 2001 From: Paul Rogers Date: Thu, 26 Jan 2023 13:24:18 -0800 Subject: [PATCH 08/43] Snapshot: went down wrong path on class loader issues --- extensions-contrib/grpc-query/pom.xml | 31 +- .../grpc/server/GrpcEndpointInitializer.java | 67 ++- .../apache/druid/grpc/server/QueryDriver.java | 447 +--------------- .../druid/grpc/server/QueryDriverImpl.java | 479 ++++++++++++++++++ .../apache/druid/grpc/server/QueryServer.java | 8 +- .../druid/grpc/server/QueryService.java | 10 +- .../druid/grpc/server/ServerBuilder.java | 47 ++ .../org/apache/druid/grpc/DriverTest.java | 6 +- .../org/apache/druid/grpc/GrpcQueryTest.java | 4 +- extensions-contrib/grpc-shaded/pom.xml | 7 + .../apache/druid/grpc/server/QueryDriver.java | 28 + .../druid/grpc/server/QueryService.java | 48 ++ .../druid/grpc/server/ServerBuilder.java | 46 ++ .../apache/druid/guice/ExtensionsLoader.java | 50 +- .../druid/guice/ExtensionsLoaderTest.java | 30 +- 15 files changed, 793 insertions(+), 515 deletions(-) create mode 100644 extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriverImpl.java create mode 100644 extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/ServerBuilder.java create mode 100644 extensions-contrib/grpc-shaded/src/main/java/org/apache/druid/grpc/server/QueryDriver.java create mode 100644 extensions-contrib/grpc-shaded/src/main/java/org/apache/druid/grpc/server/QueryService.java create mode 100644 extensions-contrib/grpc-shaded/src/main/java/org/apache/druid/grpc/server/ServerBuilder.java diff --git a/extensions-contrib/grpc-query/pom.xml b/extensions-contrib/grpc-query/pom.xml index 8df9fcf1bd20..8327d397cf5d 100644 --- a/extensions-contrib/grpc-query/pom.xml +++ b/extensions-contrib/grpc-query/pom.xml @@ -102,10 +102,39 @@ jsr305 provided + + io.netty + netty-buffer + provided + + + io.netty + netty-codec-http + provided + + + io.netty + netty-common + provided + + + io.netty + netty-handler + provided + + + io.netty + netty-resolver + provided + + + io.netty + netty-transport + provided + io.netty netty-codec-http2 - ${netty4.version} diff --git a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/GrpcEndpointInitializer.java b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/GrpcEndpointInitializer.java index 8a577f431819..6583185620f8 100644 --- a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/GrpcEndpointInitializer.java +++ b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/GrpcEndpointInitializer.java @@ -20,6 +20,7 @@ package org.apache.druid.grpc.server; import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.guice.ExtensionsLoader; import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.guice.annotations.Json; import org.apache.druid.guice.annotations.NativeQuery; @@ -31,15 +32,22 @@ import javax.inject.Inject; -import java.io.IOException; - @ManageLifecycle public class GrpcEndpointInitializer { private static final Logger log = new Logger(GrpcEndpointInitializer.class); + /** + * Name of the extension. Must match the name in the + * {@code $DRUID_HOME/extensions} directory. + */ + public static final String EXTENSION_NAME = "grpc-query"; + private final GrpcQueryConfig config; - private final QueryDriver driver; +// private final QueryDriver driver; + private final ClassLoader extnClassLoader; + private final ObjectMapper jsonMapper; + private final SqlStatementFactory sqlStatementFactory; private QueryServer server; @@ -47,35 +55,64 @@ public class GrpcEndpointInitializer public GrpcEndpointInitializer( GrpcQueryConfig config, final @Json ObjectMapper jsonMapper, - final @NativeQuery SqlStatementFactory sqlStatementFactory + final @NativeQuery SqlStatementFactory sqlStatementFactory, + final ExtensionsLoader extensionsLoader ) { this.config = config; - this.driver = new QueryDriver(jsonMapper, sqlStatementFactory); +// this.driver = new QueryDriver(jsonMapper, sqlStatementFactory); + this.jsonMapper = jsonMapper; + this.sqlStatementFactory = sqlStatementFactory; + + // Retrieve the class loader for this extension. Necessary because, in an IDE, + // the class loader for the extension's files will be the AppClassLoader since + // the IDE puts our classes on the class path. + this.extnClassLoader = extensionsLoader.getClassLoaderForExtension(EXTENSION_NAME); + if (this.extnClassLoader == null) { + throw new ISE("No extension class loader for %s: wrong name?", EXTENSION_NAME); + } } @LifecycleStart public void start() { -// String foo = io.netty.handler.codec.http2.Http2Headers.PseudoHeaderName.METHOD.name(); -// final ClassLoader oldLoader = Thread.currentThread().getContextClassLoader(); -// -// try { -// Thread.currentThread().setContextClassLoader(GrpcEndpointInitializer.class.getClassLoader()); +// ClassLoader thisLoader = getClass().getClassLoader(); + final ClassLoader oldLoader = Thread.currentThread().getContextClassLoader(); + + try { + Thread.currentThread().setContextClassLoader(extnClassLoader); +// try { +// Class found = extnClassLoader.loadClass("io.netty.handler.codec.http2.Http2Headers"); +// } catch (ClassNotFoundException e) { +// throw new ISE(e, "Fatal error: grpc query server startup failed"); +// } - server = new QueryServer(config.getPort(), driver); try { + QueryDriverImpl driver = new QueryDriverImpl(jsonMapper, sqlStatementFactory); + Class serverClass = extnClassLoader.loadClass("org.apache.druid.grpc.server.QueryServer"); + Class driverClass = extnClassLoader.loadClass("org.apache.druid.grpc.server.QueryDriver"); +// Object driver = driverClass +// .getConstructor(ObjectMapper.class, SqlStatementFactory.class) +// .newInstance(jsonMapper, sqlStatementFactory); +// Class builderClass = extnClassLoader.loadClass("org.apache.druid.grpc.server.ServerBuilder"); +// ServerBuilder builder = builderClass.getConstructor().newInstance(); +// server = builder.buildServer(config.getPort(), driver); + server = (QueryServer) serverClass + .getConstructor(Integer.class, driverClass) + .newInstance(config.getPort(), driver); + +// server = new QueryServer(config.getPort(), driver); server.start(); - } catch (IOException e) { + } catch (Exception e) { log.error(e, "Fatal error: gRPC query server startup failed"); throw new ISE(e, "Fatal error: grpc query server startup failed"); } catch (Throwable t) { log.error(t, "Fatal error: gRPC query server startup failed"); throw t; } -// } finally { -// Thread.currentThread().setContextClassLoader(oldLoader); -// } + } finally { + Thread.currentThread().setContextClassLoader(oldLoader); + } } @LifecycleStop diff --git a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriver.java b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriver.java index 74591edf1645..56e7b8718981 100644 --- a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriver.java +++ b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriver.java @@ -19,453 +19,10 @@ package org.apache.druid.grpc.server; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableMap; -import com.google.protobuf.ByteString; -import org.apache.calcite.avatica.SqlType; -import org.apache.calcite.rel.type.RelDataType; -import org.apache.druid.grpc.proto.QueryOuterClass.ColumnSchema; -import org.apache.druid.grpc.proto.QueryOuterClass.DruidType; -import org.apache.druid.grpc.proto.QueryOuterClass.QueryParameter; import org.apache.druid.grpc.proto.QueryOuterClass.QueryRequest; import org.apache.druid.grpc.proto.QueryOuterClass.QueryResponse; -import org.apache.druid.grpc.proto.QueryOuterClass.QueryResultFormat; -import org.apache.druid.grpc.proto.QueryOuterClass.QueryStatus; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.guava.Accumulator; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.server.security.Access; -import org.apache.druid.server.security.AuthenticationResult; -import org.apache.druid.server.security.ForbiddenException; -import org.apache.druid.sql.DirectStatement; -import org.apache.druid.sql.DirectStatement.ResultSet; -import org.apache.druid.sql.SqlPlanningException; -import org.apache.druid.sql.SqlQueryPlus; -import org.apache.druid.sql.SqlRowTransformer; -import org.apache.druid.sql.SqlStatementFactory; -import org.apache.druid.sql.calcite.table.RowSignatures; -import org.apache.druid.sql.http.ResultFormat; -import org.apache.druid.sql.http.SqlParameter; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.Optional; - -/** - * "Driver" for the gRPC query endpoint. Handles translating the gRPC {@link QueryRequest} - * into Druid's internal formats, running the query, and translating the results into a - * gRPC {@link QueryResponse}. Allows for easier unit testing as we separate the machinery - * of running a query, given the request, from the gRPC server machinery. - */ -public class QueryDriver +public interface QueryDriver { - /** - * Internal runtime exception to report request errors. - */ - private static class RequestError extends RuntimeException - { - public RequestError(String msg) - { - super(msg); - } - } - - private final ObjectMapper jsonMapper; - private final SqlStatementFactory sqlStatementFactory; - - public QueryDriver( - final ObjectMapper jsonMapper, - final SqlStatementFactory sqlStatementFactory - ) - { - this.jsonMapper = Preconditions.checkNotNull(jsonMapper, "jsonMapper"); - this.sqlStatementFactory = Preconditions.checkNotNull(sqlStatementFactory, "sqlStatementFactory"); - } - - /** - * First-cut synchronous query handler. Druid prefers to stream results, in - * part to avoid overly-short network timeouts. However, for now, we simply run - * the query within this call and prepare the Protobuf response. Async handling - * can come later. - */ - public QueryResponse submitQuery(QueryRequest request, AuthenticationResult authResult) - { - final SqlQueryPlus queryPlus; - try { - queryPlus = translateQuery(request, authResult); - } - catch (RuntimeException e) { - return QueryResponse.newBuilder() - .setQueryId("") - .setStatus(QueryStatus.REQUEST_ERROR) - .setErrorMessage(e.getMessage()) - .build(); - } - final DirectStatement stmt = sqlStatementFactory.directStatement(queryPlus); - final String currThreadName = Thread.currentThread().getName(); - try { - Thread.currentThread().setName(StringUtils.format("grpc-sql[%s]", stmt.sqlQueryId())); - final ResultSet thePlan = stmt.plan(); - final SqlRowTransformer rowTransformer = thePlan.createRowTransformer(); - final ByteString results = encodeResults(request.getResultFormat(), thePlan, rowTransformer); - stmt.reporter().succeeded(0); // TODO: real byte count (of payload) - stmt.close(); - return QueryResponse.newBuilder() - .setQueryId(stmt.sqlQueryId()) - .setStatus(QueryStatus.OK) - .setData(results) - .clearErrorMessage() - .addAllColumns(encodeColumns(rowTransformer)) - .build(); - } - catch (ForbiddenException e) { - stmt.reporter().failed(e); - stmt.close(); - return QueryResponse.newBuilder() - .setQueryId(stmt.sqlQueryId()) - .setStatus(QueryStatus.UNAUTHORIZED) - .setErrorMessage(Access.DEFAULT_ERROR_MESSAGE) - .build(); - } - catch (RequestError e) { - stmt.reporter().failed(e); - stmt.close(); - return QueryResponse.newBuilder() - .setQueryId(stmt.sqlQueryId()) - .setStatus(QueryStatus.REQUEST_ERROR) - .setErrorMessage(e.getMessage()) - .build(); - } - catch (SqlPlanningException e) { - stmt.reporter().failed(e); - stmt.close(); - return QueryResponse.newBuilder() - .setQueryId(stmt.sqlQueryId()) - .setStatus(QueryStatus.INVALID_SQL) - .setErrorMessage(e.getMessage()) - .build(); - } - catch (IOException | RuntimeException e) { - stmt.reporter().failed(e); - stmt.close(); - return QueryResponse.newBuilder() - .setQueryId(stmt.sqlQueryId()) - .setStatus(QueryStatus.RUNTIME_ERROR) - .setErrorMessage(e.getMessage()) - .build(); - } - // There is a claim that Calcite sometimes throws a java.lang.AssertionError, but we do not have a test that can - // reproduce it checked into the code (the best we have is something that uses mocks to throw an Error, which is - // dubious at best). We keep this just in case, but it might be best to remove it and see where the - // AssertionErrors are coming from and do something to ensure that they don't actually make it out of Calcite - catch (AssertionError e) { - stmt.reporter().failed(e); - stmt.close(); - return QueryResponse.newBuilder() - .setQueryId(stmt.sqlQueryId()) - .setStatus(QueryStatus.RUNTIME_ERROR) - .setErrorMessage(e.getMessage()) - .build(); - } - finally { - Thread.currentThread().setName(currThreadName); - } - } - - /** - * Convert the rRPC query format to the internal {@link SqlQueryPlus} format. - */ - private SqlQueryPlus translateQuery(QueryRequest request, AuthenticationResult authResult) - { - return SqlQueryPlus.builder() - .sql(request.getQuery()) - .context(translateContext(request)) - .sqlParameters(translateParameters(request)) - .auth(authResult) - .build(); - } - - /** - * Translate the query context from the gRPC format to the internal format. When - * read from REST/JSON, the JSON translator will convert the type of each value - * into a number, Boolean, etc. gRPC has no similar feature. Rather than clutter up - * the gRPC request with typed context values, we rely on the existing code that can - * translate string values to the desired type on the fly. Thus, we build up a - * {@code Map}. - */ - private Map translateContext(QueryRequest request) - { - ImmutableMap.Builder builder = ImmutableMap.builder(); - if (request.getContextCount() > 0) { - for (Map.Entry entry : request.getContextMap().entrySet()) { - builder.put(entry.getKey(), entry.getValue()); - } - } - return builder.build(); - } - - /** - * Convert the gRPC parameter format to the internal Druid {@link SqlParameter} - * format. That format is then again translated by the {@link SqlQueryPlus} class. - */ - private List translateParameters(QueryRequest request) - { - if (request.getParametersCount() == 0) { - return null; - } - List params = new ArrayList<>(); - for (QueryParameter value : request.getParametersList()) { - params.add(translateParameter(value)); - } - return params; - } - - private SqlParameter translateParameter(QueryParameter value) - { - switch (value.getValueCase()) { - case ARRAYVALUE: - // Not yet supported: waiting for an open PR - return null; - case DOUBLEVALUE: - return new SqlParameter(SqlType.DOUBLE, value.getDoubleValue()); - case LONGVALUE: - return new SqlParameter(SqlType.BIGINT, value.getLongValue()); - case STRINGVALUE: - return new SqlParameter(SqlType.VARCHAR, value.getStringValue()); - case NULLVALUE: - case VALUE_NOT_SET: - return null; - default: - throw new RequestError("Invalid parameter type: " + value.getValueCase().name()); - } - } - - /** - * Translate the column schema from the Druid internal form to the gRPC - * {@link ColumnSchema} form. Note that since the gRPC response returns the - * schema, none of the data formats include a header. This makes the data format - * simpler and cleaner. - */ - private Iterable encodeColumns(SqlRowTransformer rowTransformer) - { - RelDataType rowType = rowTransformer.getRowType(); - final RowSignature signature = RowSignatures.fromRelDataType(rowType.getFieldNames(), rowType); - List cols = new ArrayList<>(); - for (int i = 0; i < rowType.getFieldCount(); i++) { - ColumnSchema col = ColumnSchema.newBuilder() - .setName(signature.getColumnName(i)) - .setSqlType(rowType.getFieldList().get(i).getType().getSqlTypeName().getName()) - .setDruidType(convertDruidType(signature.getColumnType(i))) - .build(); - cols.add(col); - } - return cols; - } - - /** - * Convert from Druid's internal format of the Druid data type to the gRPC form. - */ - private DruidType convertDruidType(Optional colType) - { - if (!colType.isPresent()) { - return DruidType.UNKNOWN_TYPE; - } - ColumnType druidType = colType.get(); - if (druidType == ColumnType.STRING) { - return DruidType.STRING; - } - if (druidType == ColumnType.STRING_ARRAY) { - return DruidType.STRING_ARRAY; - } - if (druidType == ColumnType.LONG) { - return DruidType.LONG; - } - if (druidType == ColumnType.LONG_ARRAY) { - return DruidType.LONG_ARRAY; - } - if (druidType == ColumnType.FLOAT) { - return DruidType.FLOAT; - } - if (druidType == ColumnType.FLOAT_ARRAY) { - return DruidType.FLOAT_ARRAY; - } - if (druidType == ColumnType.DOUBLE) { - return DruidType.DOUBLE; - } - if (druidType == ColumnType.DOUBLE_ARRAY) { - return DruidType.DOUBLE_ARRAY; - } - if (druidType == ColumnType.UNKNOWN_COMPLEX) { - return DruidType.COMPLEX; - } - return DruidType.UNKNOWN_TYPE; - } - - /** - * Generic mechanism to write query results to one of the supported gRPC formats. - */ - public interface GrpcResultWriter - { - void start() throws IOException; - void writeRow(Object[] row) throws IOException; - void close() throws IOException; - } - - /** - * Writer for the SQL result formats. Reuses the SQL format writer implementations. - * Note: gRPC does not use the headers: schema information is available in the - * rRPC response. - */ - public static class GrpcResultFormatWriter implements GrpcResultWriter - { - protected final ResultFormat.Writer formatWriter; - protected final SqlRowTransformer rowTransformer; - - public GrpcResultFormatWriter( - final ResultFormat.Writer formatWriter, - final SqlRowTransformer rowTransformer - ) - { - this.formatWriter = formatWriter; - this.rowTransformer = rowTransformer; - } - - @Override - public void start() throws IOException - { - } - - @Override - public void writeRow(Object[] row) throws IOException - { - formatWriter.writeRowStart(); - for (int i = 0; i < rowTransformer.getFieldList().size(); i++) { - final Object value = rowTransformer.transform(row, i); - formatWriter.writeRowField(rowTransformer.getFieldList().get(i), value); - } - formatWriter.writeRowEnd(); - } - - @Override - public void close() throws IOException - { - formatWriter.close(); - } - } - - /** - * Internal runtime exception to pass {@link IOException}s though the - * {@link Sequence} {@link Accumulator} protocol. - */ - private static class ResponseError extends RuntimeException - { - public ResponseError(IOException e) - { - super(e); - } - } - - /** - * Druid query results use a complex {@link Sequence} mechanism. This class uses an - * {@link Accumulator} to walk the results and present each to the associated - * results writer. This is a very rough analogy of the {@code SqlResourceQueryResultPusher} - * in the REST {@code SqlResource} class. - */ - public static class GrpcResultsAccumulator implements Accumulator - { - private final GrpcResultWriter writer; - - public GrpcResultsAccumulator(final GrpcResultWriter writer) - { - this.writer = writer; - } - - public void push(org.apache.druid.server.QueryResponse queryResponse) throws IOException - { - final Sequence results = queryResponse.getResults(); - writer.start(); - try { - results.accumulate(null, this); - } - catch (ResponseError e) { - throw (IOException) e.getCause(); - } - writer.close(); - } - - @Override - public Void accumulate(Void accumulated, Object[] in) - { - try { - writer.writeRow(in); - } - catch (IOException e) { - throw new ResponseError(e); - } - return null; - } - } - - /** - * Convert the query results to a set of bytes to be attached to the query response. - *

- * This version is pretty basic: the results are materialized as a byte array. That's - * fine for small result sets, but should be rethought for larger result sets. - */ - private ByteString encodeResults( - final QueryResultFormat queryResultFormat, - final ResultSet thePlan, - final SqlRowTransformer rowTransformer - ) throws IOException - { - // Accumulate the results as a byte array. - ByteArrayOutputStream out = new ByteArrayOutputStream(); - GrpcResultWriter writer; - - // For the SQL-supported formats, use the SQL-provided writers. - switch (queryResultFormat) { - case CSV: - writer = new GrpcResultFormatWriter( - ResultFormat.CSV.createFormatter(out, jsonMapper), - rowTransformer - ); - break; - case JSON_ARRAY: - writer = new GrpcResultFormatWriter( - ResultFormat.ARRAY.createFormatter(out, jsonMapper), - rowTransformer - ); - break; - case JSON_ARRAY_LINES: - writer = new GrpcResultFormatWriter( - ResultFormat.ARRAYLINES.createFormatter(out, jsonMapper), - rowTransformer - ); - break; - - // TODO: Provide additional writers for the other formats which we - // want in gRPC. - case JSON_OBJECT: - throw new UnsupportedOperationException(); // TODO - case JSON_OBJECT_LINES: - throw new UnsupportedOperationException(); // TODO - case PROTOBUF_INLINE: - throw new UnsupportedOperationException(); // TODO - - // This is the hard one: encode the results as a Protobuf array. - case PROTOBUF_RESPONSE: - throw new UnsupportedOperationException(); // TODO - default: - throw new RequestError("Unsupported query result format"); - } - GrpcResultsAccumulator accumulator = new GrpcResultsAccumulator(writer); - accumulator.push(thePlan.run()); - return ByteString.copyFrom(out.toByteArray()); - } + QueryResponse submitQuery(QueryRequest request); } diff --git a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriverImpl.java b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriverImpl.java new file mode 100644 index 000000000000..79cbb7875440 --- /dev/null +++ b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriverImpl.java @@ -0,0 +1,479 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.grpc.server; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableMap; +import com.google.protobuf.ByteString; +import org.apache.calcite.avatica.SqlType; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.druid.grpc.proto.QueryOuterClass.ColumnSchema; +import org.apache.druid.grpc.proto.QueryOuterClass.DruidType; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryParameter; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryRequest; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryResponse; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryResultFormat; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryStatus; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.guava.Accumulator; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.server.security.Access; +import org.apache.druid.server.security.AuthConfig; +import org.apache.druid.server.security.AuthenticationResult; +import org.apache.druid.server.security.ForbiddenException; +import org.apache.druid.sql.DirectStatement; +import org.apache.druid.sql.DirectStatement.ResultSet; +import org.apache.druid.sql.SqlPlanningException; +import org.apache.druid.sql.SqlQueryPlus; +import org.apache.druid.sql.SqlRowTransformer; +import org.apache.druid.sql.SqlStatementFactory; +import org.apache.druid.sql.calcite.table.RowSignatures; +import org.apache.druid.sql.http.ResultFormat; +import org.apache.druid.sql.http.SqlParameter; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +/** + * "Driver" for the gRPC query endpoint. Handles translating the gRPC {@link QueryRequest} + * into Druid's internal formats, running the query, and translating the results into a + * gRPC {@link QueryResponse}. Allows for easier unit testing as we separate the machinery + * of running a query, given the request, from the gRPC server machinery. + */ +public class QueryDriverImpl implements QueryDriver +{ + /** + * Internal runtime exception to report request errors. + */ + private static class RequestError extends RuntimeException + { + public RequestError(String msg) + { + super(msg); + } + } + + private final ObjectMapper jsonMapper; + private final SqlStatementFactory sqlStatementFactory; + + public QueryDriverImpl( + final ObjectMapper jsonMapper, + final SqlStatementFactory sqlStatementFactory + ) + { + this.jsonMapper = Preconditions.checkNotNull(jsonMapper, "jsonMapper"); + this.sqlStatementFactory = Preconditions.checkNotNull(sqlStatementFactory, "sqlStatementFactory"); + } + + /** + * First-cut synchronous query handler. Druid prefers to stream results, in + * part to avoid overly-short network timeouts. However, for now, we simply run + * the query within this call and prepare the Protobuf response. Async handling + * can come later. + */ + @Override + public QueryResponse submitQuery(QueryRequest request) + { + // TODO: How will we get the auth result for gRPC? + AuthenticationResult authResult = new AuthenticationResult( + "superUser", + AuthConfig.ALLOW_ALL_NAME, + null, null + ); + final SqlQueryPlus queryPlus; + try { + queryPlus = translateQuery(request, authResult); + } + catch (RuntimeException e) { + return QueryResponse.newBuilder() + .setQueryId("") + .setStatus(QueryStatus.REQUEST_ERROR) + .setErrorMessage(e.getMessage()) + .build(); + } + final DirectStatement stmt = sqlStatementFactory.directStatement(queryPlus); + final String currThreadName = Thread.currentThread().getName(); + try { + Thread.currentThread().setName(StringUtils.format("grpc-sql[%s]", stmt.sqlQueryId())); + final ResultSet thePlan = stmt.plan(); + final SqlRowTransformer rowTransformer = thePlan.createRowTransformer(); + final ByteString results = encodeResults(request.getResultFormat(), thePlan, rowTransformer); + stmt.reporter().succeeded(0); // TODO: real byte count (of payload) + stmt.close(); + return QueryResponse.newBuilder() + .setQueryId(stmt.sqlQueryId()) + .setStatus(QueryStatus.OK) + .setData(results) + .clearErrorMessage() + .addAllColumns(encodeColumns(rowTransformer)) + .build(); + } + catch (ForbiddenException e) { + stmt.reporter().failed(e); + stmt.close(); + return QueryResponse.newBuilder() + .setQueryId(stmt.sqlQueryId()) + .setStatus(QueryStatus.UNAUTHORIZED) + .setErrorMessage(Access.DEFAULT_ERROR_MESSAGE) + .build(); + } + catch (RequestError e) { + stmt.reporter().failed(e); + stmt.close(); + return QueryResponse.newBuilder() + .setQueryId(stmt.sqlQueryId()) + .setStatus(QueryStatus.REQUEST_ERROR) + .setErrorMessage(e.getMessage()) + .build(); + } + catch (SqlPlanningException e) { + stmt.reporter().failed(e); + stmt.close(); + return QueryResponse.newBuilder() + .setQueryId(stmt.sqlQueryId()) + .setStatus(QueryStatus.INVALID_SQL) + .setErrorMessage(e.getMessage()) + .build(); + } + catch (IOException | RuntimeException e) { + stmt.reporter().failed(e); + stmt.close(); + return QueryResponse.newBuilder() + .setQueryId(stmt.sqlQueryId()) + .setStatus(QueryStatus.RUNTIME_ERROR) + .setErrorMessage(e.getMessage()) + .build(); + } + // There is a claim that Calcite sometimes throws a java.lang.AssertionError, but we do not have a test that can + // reproduce it checked into the code (the best we have is something that uses mocks to throw an Error, which is + // dubious at best). We keep this just in case, but it might be best to remove it and see where the + // AssertionErrors are coming from and do something to ensure that they don't actually make it out of Calcite + catch (AssertionError e) { + stmt.reporter().failed(e); + stmt.close(); + return QueryResponse.newBuilder() + .setQueryId(stmt.sqlQueryId()) + .setStatus(QueryStatus.RUNTIME_ERROR) + .setErrorMessage(e.getMessage()) + .build(); + } + finally { + Thread.currentThread().setName(currThreadName); + } + } + + /** + * Convert the rRPC query format to the internal {@link SqlQueryPlus} format. + */ + private SqlQueryPlus translateQuery(QueryRequest request, AuthenticationResult authResult) + { + return SqlQueryPlus.builder() + .sql(request.getQuery()) + .context(translateContext(request)) + .sqlParameters(translateParameters(request)) + .auth(authResult) + .build(); + } + + /** + * Translate the query context from the gRPC format to the internal format. When + * read from REST/JSON, the JSON translator will convert the type of each value + * into a number, Boolean, etc. gRPC has no similar feature. Rather than clutter up + * the gRPC request with typed context values, we rely on the existing code that can + * translate string values to the desired type on the fly. Thus, we build up a + * {@code Map}. + */ + private Map translateContext(QueryRequest request) + { + ImmutableMap.Builder builder = ImmutableMap.builder(); + if (request.getContextCount() > 0) { + for (Map.Entry entry : request.getContextMap().entrySet()) { + builder.put(entry.getKey(), entry.getValue()); + } + } + return builder.build(); + } + + /** + * Convert the gRPC parameter format to the internal Druid {@link SqlParameter} + * format. That format is then again translated by the {@link SqlQueryPlus} class. + */ + private List translateParameters(QueryRequest request) + { + if (request.getParametersCount() == 0) { + return null; + } + List params = new ArrayList<>(); + for (QueryParameter value : request.getParametersList()) { + params.add(translateParameter(value)); + } + return params; + } + + private SqlParameter translateParameter(QueryParameter value) + { + switch (value.getValueCase()) { + case ARRAYVALUE: + // Not yet supported: waiting for an open PR + return null; + case DOUBLEVALUE: + return new SqlParameter(SqlType.DOUBLE, value.getDoubleValue()); + case LONGVALUE: + return new SqlParameter(SqlType.BIGINT, value.getLongValue()); + case STRINGVALUE: + return new SqlParameter(SqlType.VARCHAR, value.getStringValue()); + case NULLVALUE: + case VALUE_NOT_SET: + return null; + default: + throw new RequestError("Invalid parameter type: " + value.getValueCase().name()); + } + } + + /** + * Translate the column schema from the Druid internal form to the gRPC + * {@link ColumnSchema} form. Note that since the gRPC response returns the + * schema, none of the data formats include a header. This makes the data format + * simpler and cleaner. + */ + private Iterable encodeColumns(SqlRowTransformer rowTransformer) + { + RelDataType rowType = rowTransformer.getRowType(); + final RowSignature signature = RowSignatures.fromRelDataType(rowType.getFieldNames(), rowType); + List cols = new ArrayList<>(); + for (int i = 0; i < rowType.getFieldCount(); i++) { + ColumnSchema col = ColumnSchema.newBuilder() + .setName(signature.getColumnName(i)) + .setSqlType(rowType.getFieldList().get(i).getType().getSqlTypeName().getName()) + .setDruidType(convertDruidType(signature.getColumnType(i))) + .build(); + cols.add(col); + } + return cols; + } + + /** + * Convert from Druid's internal format of the Druid data type to the gRPC form. + */ + private DruidType convertDruidType(Optional colType) + { + if (!colType.isPresent()) { + return DruidType.UNKNOWN_TYPE; + } + ColumnType druidType = colType.get(); + if (druidType == ColumnType.STRING) { + return DruidType.STRING; + } + if (druidType == ColumnType.STRING_ARRAY) { + return DruidType.STRING_ARRAY; + } + if (druidType == ColumnType.LONG) { + return DruidType.LONG; + } + if (druidType == ColumnType.LONG_ARRAY) { + return DruidType.LONG_ARRAY; + } + if (druidType == ColumnType.FLOAT) { + return DruidType.FLOAT; + } + if (druidType == ColumnType.FLOAT_ARRAY) { + return DruidType.FLOAT_ARRAY; + } + if (druidType == ColumnType.DOUBLE) { + return DruidType.DOUBLE; + } + if (druidType == ColumnType.DOUBLE_ARRAY) { + return DruidType.DOUBLE_ARRAY; + } + if (druidType == ColumnType.UNKNOWN_COMPLEX) { + return DruidType.COMPLEX; + } + return DruidType.UNKNOWN_TYPE; + } + + /** + * Generic mechanism to write query results to one of the supported gRPC formats. + */ + public interface GrpcResultWriter + { + void start() throws IOException; + void writeRow(Object[] row) throws IOException; + void close() throws IOException; + } + + /** + * Writer for the SQL result formats. Reuses the SQL format writer implementations. + * Note: gRPC does not use the headers: schema information is available in the + * rRPC response. + */ + public static class GrpcResultFormatWriter implements GrpcResultWriter + { + protected final ResultFormat.Writer formatWriter; + protected final SqlRowTransformer rowTransformer; + + public GrpcResultFormatWriter( + final ResultFormat.Writer formatWriter, + final SqlRowTransformer rowTransformer + ) + { + this.formatWriter = formatWriter; + this.rowTransformer = rowTransformer; + } + + @Override + public void start() throws IOException + { + } + + @Override + public void writeRow(Object[] row) throws IOException + { + formatWriter.writeRowStart(); + for (int i = 0; i < rowTransformer.getFieldList().size(); i++) { + final Object value = rowTransformer.transform(row, i); + formatWriter.writeRowField(rowTransformer.getFieldList().get(i), value); + } + formatWriter.writeRowEnd(); + } + + @Override + public void close() throws IOException + { + formatWriter.close(); + } + } + + /** + * Internal runtime exception to pass {@link IOException}s though the + * {@link Sequence} {@link Accumulator} protocol. + */ + private static class ResponseError extends RuntimeException + { + public ResponseError(IOException e) + { + super(e); + } + } + + /** + * Druid query results use a complex {@link Sequence} mechanism. This class uses an + * {@link Accumulator} to walk the results and present each to the associated + * results writer. This is a very rough analogy of the {@code SqlResourceQueryResultPusher} + * in the REST {@code SqlResource} class. + */ + public static class GrpcResultsAccumulator implements Accumulator + { + private final GrpcResultWriter writer; + + public GrpcResultsAccumulator(final GrpcResultWriter writer) + { + this.writer = writer; + } + + public void push(org.apache.druid.server.QueryResponse queryResponse) throws IOException + { + final Sequence results = queryResponse.getResults(); + writer.start(); + try { + results.accumulate(null, this); + } + catch (ResponseError e) { + throw (IOException) e.getCause(); + } + writer.close(); + } + + @Override + public Void accumulate(Void accumulated, Object[] in) + { + try { + writer.writeRow(in); + } + catch (IOException e) { + throw new ResponseError(e); + } + return null; + } + } + + /** + * Convert the query results to a set of bytes to be attached to the query response. + *

+ * This version is pretty basic: the results are materialized as a byte array. That's + * fine for small result sets, but should be rethought for larger result sets. + */ + private ByteString encodeResults( + final QueryResultFormat queryResultFormat, + final ResultSet thePlan, + final SqlRowTransformer rowTransformer + ) throws IOException + { + // Accumulate the results as a byte array. + ByteArrayOutputStream out = new ByteArrayOutputStream(); + GrpcResultWriter writer; + + // For the SQL-supported formats, use the SQL-provided writers. + switch (queryResultFormat) { + case CSV: + writer = new GrpcResultFormatWriter( + ResultFormat.CSV.createFormatter(out, jsonMapper), + rowTransformer + ); + break; + case JSON_ARRAY: + writer = new GrpcResultFormatWriter( + ResultFormat.ARRAY.createFormatter(out, jsonMapper), + rowTransformer + ); + break; + case JSON_ARRAY_LINES: + writer = new GrpcResultFormatWriter( + ResultFormat.ARRAYLINES.createFormatter(out, jsonMapper), + rowTransformer + ); + break; + + // TODO: Provide additional writers for the other formats which we + // want in gRPC. + case JSON_OBJECT: + throw new UnsupportedOperationException(); // TODO + case JSON_OBJECT_LINES: + throw new UnsupportedOperationException(); // TODO + case PROTOBUF_INLINE: + throw new UnsupportedOperationException(); // TODO + + // This is the hard one: encode the results as a Protobuf array. + case PROTOBUF_RESPONSE: + throw new UnsupportedOperationException(); // TODO + default: + throw new RequestError("Unsupported query result format"); + } + GrpcResultsAccumulator accumulator = new GrpcResultsAccumulator(writer); + accumulator.push(thePlan.run()); + return ByteString.copyFrom(out.toByteArray()); + } +} diff --git a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryServer.java b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryServer.java index 7470fac90c75..c8784b254f4b 100644 --- a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryServer.java +++ b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryServer.java @@ -25,13 +25,14 @@ import org.apache.druid.java.util.common.logger.Logger; import javax.inject.Inject; + import java.io.IOException; import java.util.concurrent.TimeUnit; /** * Basic gRPC server adapted from the gRPC examples. Delegates to the - * {@link QueryDriver} class to do the actual work of running the query. + * {@link QueryDriverImpl} class to do the actual work of running the query. *

* This class is preliminary. It is good enough for unit tests, but a bit more work * is needed to integrate this class into the Druid server. @@ -47,7 +48,10 @@ public class QueryServer private Server server; @Inject - public QueryServer(int port, QueryDriver driver) + public QueryServer( + Integer port, + QueryDriver driver + ) { this.port = port; this.driver = driver; diff --git a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryService.java b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryService.java index 54ae92d8d3f5..19e2cf1b18a4 100644 --- a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryService.java +++ b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryService.java @@ -23,8 +23,6 @@ import org.apache.druid.grpc.proto.QueryGrpc; import org.apache.druid.grpc.proto.QueryOuterClass.QueryRequest; import org.apache.druid.grpc.proto.QueryOuterClass.QueryResponse; -import org.apache.druid.server.security.AuthConfig; -import org.apache.druid.server.security.AuthenticationResult; /** * Implementation of the gRPC Query service. Provides a single method @@ -43,13 +41,7 @@ public QueryService(QueryDriver driver) @Override public void submitQuery(QueryRequest request, StreamObserver responseObserver) { - // TODO: How will we get the auth result for gRPC? - AuthenticationResult authResult = new AuthenticationResult( - "superUser", - AuthConfig.ALLOW_ALL_NAME, - null, null - ); - QueryResponse reply = driver.submitQuery(request, authResult); + QueryResponse reply = driver.submitQuery(request); responseObserver.onNext(reply); responseObserver.onCompleted(); } diff --git a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/ServerBuilder.java b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/ServerBuilder.java new file mode 100644 index 000000000000..87dd878e1437 --- /dev/null +++ b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/ServerBuilder.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.grpc.server; + +import io.grpc.BindableService; +import io.grpc.Grpc; +import io.grpc.InsecureServerCredentials; +import io.grpc.Server; + +/** + * Builds the Grpc Server given a port. This builds a Netty-based server + * internally. Done in the shaded jar so that this class can be created + * using the extension class loader, so that the gRPC classes are also + * created using that class loader which allows gRPC to find its components. + * The rest of the server is created implicitly which will use the + * the extension class loader in production, the app loader when debugging. + * If you find you are getting errors about classes not found, or arguments + * not matching, it is because you have instances created in the app class + * loader which don't have visibility to classes in this extension. + */ +public class ServerBuilder +{ + public Server buildServer(int port, QueryDriver driver) + { + QueryService service = new QueryServer(port, driver); + return Grpc.newServerBuilderForPort(port, InsecureServerCredentials.create()) + .addService(service) + .build(); + } +} diff --git a/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/DriverTest.java b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/DriverTest.java index 98fa27acd617..07316b47b83c 100644 --- a/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/DriverTest.java +++ b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/DriverTest.java @@ -25,7 +25,7 @@ import org.apache.druid.grpc.proto.QueryOuterClass.QueryResponse; import org.apache.druid.grpc.proto.QueryOuterClass.QueryResultFormat; import org.apache.druid.grpc.proto.QueryOuterClass.QueryStatus; -import org.apache.druid.grpc.server.QueryDriver; +import org.apache.druid.grpc.server.QueryDriverImpl; import org.apache.druid.sql.calcite.util.CalciteTests; import org.junit.BeforeClass; import org.junit.ClassRule; @@ -45,13 +45,13 @@ public class DriverTest @ClassRule public static TemporaryFolder temporaryFolder = new TemporaryFolder(); private static QueryFrameworkFixture frameworkFixture; - private static QueryDriver driver; + private static QueryDriverImpl driver; @BeforeClass public static void setup() throws IOException { frameworkFixture = new QueryFrameworkFixture(temporaryFolder.newFolder()); - driver = new QueryDriver( + driver = new QueryDriverImpl( frameworkFixture.jsonMapper(), frameworkFixture.statementFactory() ); diff --git a/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTest.java b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTest.java index 7ef0ac292690..7f207675850f 100644 --- a/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTest.java +++ b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTest.java @@ -28,7 +28,7 @@ import org.apache.druid.grpc.proto.QueryOuterClass.QueryResponse; import org.apache.druid.grpc.proto.QueryOuterClass.QueryResultFormat; import org.apache.druid.grpc.proto.QueryOuterClass.QueryStatus; -import org.apache.druid.grpc.server.QueryDriver; +import org.apache.druid.grpc.server.QueryDriverImpl; import org.apache.druid.grpc.server.QueryServer; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -82,7 +82,7 @@ public void close() throws InterruptedException public static void setup() throws IOException { frameworkFixture = new QueryFrameworkFixture(temporaryFolder.newFolder()); - QueryDriver driver = new QueryDriver( + QueryDriverImpl driver = new QueryDriverImpl( frameworkFixture.jsonMapper(), frameworkFixture.statementFactory() ); diff --git a/extensions-contrib/grpc-shaded/pom.xml b/extensions-contrib/grpc-shaded/pom.xml index 8a2bb0497eb1..5b5e211da84a 100644 --- a/extensions-contrib/grpc-shaded/pom.xml +++ b/extensions-contrib/grpc-shaded/pom.xml @@ -53,6 +53,13 @@ + + + io.grpc + grpc-core + io.grpc grpc-api diff --git a/extensions-contrib/grpc-shaded/src/main/java/org/apache/druid/grpc/server/QueryDriver.java b/extensions-contrib/grpc-shaded/src/main/java/org/apache/druid/grpc/server/QueryDriver.java new file mode 100644 index 000000000000..56e7b8718981 --- /dev/null +++ b/extensions-contrib/grpc-shaded/src/main/java/org/apache/druid/grpc/server/QueryDriver.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.grpc.server; + +import org.apache.druid.grpc.proto.QueryOuterClass.QueryRequest; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryResponse; + +public interface QueryDriver +{ + QueryResponse submitQuery(QueryRequest request); +} diff --git a/extensions-contrib/grpc-shaded/src/main/java/org/apache/druid/grpc/server/QueryService.java b/extensions-contrib/grpc-shaded/src/main/java/org/apache/druid/grpc/server/QueryService.java new file mode 100644 index 000000000000..19e2cf1b18a4 --- /dev/null +++ b/extensions-contrib/grpc-shaded/src/main/java/org/apache/druid/grpc/server/QueryService.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.grpc.server; + +import io.grpc.stub.StreamObserver; +import org.apache.druid.grpc.proto.QueryGrpc; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryRequest; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryResponse; + +/** + * Implementation of the gRPC Query service. Provides a single method + * to run a query using the "driver" that holds the actual Druid SQL + * logic. + */ +class QueryService extends QueryGrpc.QueryImplBase +{ + private final QueryDriver driver; + + public QueryService(QueryDriver driver) + { + this.driver = driver; + } + + @Override + public void submitQuery(QueryRequest request, StreamObserver responseObserver) + { + QueryResponse reply = driver.submitQuery(request); + responseObserver.onNext(reply); + responseObserver.onCompleted(); + } +} diff --git a/extensions-contrib/grpc-shaded/src/main/java/org/apache/druid/grpc/server/ServerBuilder.java b/extensions-contrib/grpc-shaded/src/main/java/org/apache/druid/grpc/server/ServerBuilder.java new file mode 100644 index 000000000000..5ab34d2cd5e0 --- /dev/null +++ b/extensions-contrib/grpc-shaded/src/main/java/org/apache/druid/grpc/server/ServerBuilder.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.grpc.server; + +import io.grpc.BindableService; +import io.grpc.Grpc; +import io.grpc.InsecureServerCredentials; +import io.grpc.Server; + +/** + * Builds the Grpc Server given a port. This builds a Netty-based server + * internally. Done in the shaded jar so that this class can be created + * using the extension class loader, so that the gRPC classes are also + * created using that class loader which allows gRPC to find its components. + * The rest of the server is created implicitly which will use the + * the extension class loader in production, the app loader when debugging. + * If you find you are getting errors about classes not found, or arguments + * not matching, it is because you have instances created in the app class + * loader which don't have visibility to classes in this extension. + */ +public class ServerBuilder +{ + public Server buildServer(int port, BindableService service) + { + return Grpc.newServerBuilderForPort(port, InsecureServerCredentials.create()) + .addService(service) + .build(); + } +} diff --git a/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java b/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java index 0d3b035e7a6a..172a5dffc2c4 100644 --- a/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java +++ b/processing/src/main/java/org/apache/druid/guice/ExtensionsLoader.java @@ -24,7 +24,6 @@ import org.apache.commons.io.FileUtils; import org.apache.druid.initialization.DruidModule; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.logger.Logger; import javax.inject.Inject; @@ -54,6 +53,12 @@ * extensions during initialization. The design, however, should support * any kind of extension that may be needed in the future. * The extensions are cached so that they can be reported by various REST APIs. + *

+ * Extensions reside in a directory. The name of the directory is the extension + * name. No two extensions can have the same name. This is not actually a restriction + * as extensions reside in {@code $DRUID_HOME/extensions}, so each extension must + * be in its own extension. The extension name is the same as that used in the + * Druid extension load list. */ @LazySingleton public class ExtensionsLoader @@ -61,7 +66,7 @@ public class ExtensionsLoader private static final Logger log = new Logger(ExtensionsLoader.class); private final ExtensionsConfig extensionsConfig; - private final ConcurrentHashMap, URLClassLoader> loaders = new ConcurrentHashMap<>(); + private final ConcurrentHashMap loaders = new ConcurrentHashMap<>(); /** * Map of loaded extensions, keyed by class (or interface). @@ -109,7 +114,7 @@ public Collection getLoadedModules() } @VisibleForTesting - public Map, URLClassLoader> getLoadersMap() + public Map getLoadersMap() { return loaders; } @@ -195,15 +200,20 @@ public File[] getExtensionFilesToLoad() * * @return a URLClassLoader that loads all the jars on which the extension is dependent */ + public URLClassLoader getClassLoaderForExtension(File extension) + { + return getClassLoaderForExtension(extension, extensionsConfig.isUseExtensionClassloaderFirst()); + } + public URLClassLoader getClassLoaderForExtension(File extension, boolean useExtensionClassloaderFirst) { return loaders.computeIfAbsent( - Pair.of(extension, useExtensionClassloaderFirst), - k -> makeClassLoaderForExtension(k.lhs, k.rhs) + extension.getName(), + k -> makeClassLoaderForExtension(extension, useExtensionClassloaderFirst) ); } - private static URLClassLoader makeClassLoaderForExtension( + private URLClassLoader makeClassLoaderForExtension( final File extension, final boolean useExtensionClassloaderFirst ) @@ -226,10 +236,31 @@ private static URLClassLoader makeClassLoaderForExtension( if (useExtensionClassloaderFirst) { return new ExtensionFirstClassLoader(urls, ExtensionsLoader.class.getClassLoader()); } else { - return new URLClassLoader(urls, ExtensionsLoader.class.getClassLoader()); + return new URLClassLoader( + // extension.getName() - after moving off of Java 8 + urls, + ExtensionsLoader.class.getClassLoader() + ); } } + /** + * Obtains the class loader for the given extension name. Assumes the extension + * has been loaded. This method primarily solves the use case of resolving a class + * within jars in an extension, when run from an IDE. When run in production, the + * class loader of the extensions {@code DruidModule} will give the extension class + * loader. But, when run in an IDE, the extension may be on the class path, so that + * the {@code AppClassLoader} finds the class before the extension class loader does. + * In this case, classes from the extension report the {@code AppClassLoader}, not + * the extensions {@link URLClassLoader} as their class loader. When this happens, + * an attempt to find a class using the class loader will fail. This method is a workaround: + * even in an IDE, we use the actual extension class loader. + */ + public URLClassLoader getClassLoaderForExtension(String extensionName) + { + return loaders.get(extensionName); + } + public static List getURLsForClasspath(String cp) { try { @@ -293,10 +324,7 @@ private void addAllFromFileSystem() for (File extension : getExtensionFilesToLoad()) { log.debug("Loading extension [%s] for class [%s]", extension.getName(), serviceClass); try { - final URLClassLoader loader = getClassLoaderForExtension( - extension, - extensionsConfig.isUseExtensionClassloaderFirst() - ); + final URLClassLoader loader = getClassLoaderForExtension(extension); log.info( "Loading extension [%s], jars: %s", diff --git a/processing/src/test/java/org/apache/druid/guice/ExtensionsLoaderTest.java b/processing/src/test/java/org/apache/druid/guice/ExtensionsLoaderTest.java index 44b7f06fb3d8..07c604a95a74 100644 --- a/processing/src/test/java/org/apache/druid/guice/ExtensionsLoaderTest.java +++ b/processing/src/test/java/org/apache/druid/guice/ExtensionsLoaderTest.java @@ -25,7 +25,6 @@ import com.google.inject.Injector; import org.apache.druid.initialization.DruidModule; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Pair; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -74,7 +73,7 @@ public void test04DuplicateClassLoaderExtensions() throws Exception Injector startupInjector = startupInjector(); ExtensionsLoader extnLoader = ExtensionsLoader.instance(startupInjector); - Pair key = Pair.of(extensionDir, true); + String key = extensionDir.getName(); extnLoader.getLoadersMap() .put(key, new URLClassLoader(new URL[]{}, ExtensionsLoader.class.getClassLoader())); @@ -99,11 +98,12 @@ public void test06GetClassLoaderForExtension() throws IOException a_jar.createNewFile(); b_jar.createNewFile(); c_jar.createNewFile(); - final URLClassLoader loader = extnLoader.getClassLoaderForExtension(some_extension_dir, false); + final URLClassLoader loader = extnLoader.getClassLoaderForExtension(some_extension_dir); final URL[] expectedURLs = new URL[]{a_jar.toURI().toURL(), b_jar.toURI().toURL(), c_jar.toURI().toURL()}; final URL[] actualURLs = loader.getURLs(); Arrays.sort(actualURLs, Comparator.comparing(URL::getPath)); Assert.assertArrayEquals(expectedURLs, actualURLs); + Assert.assertSame(loader, extnLoader.getClassLoaderForExtension(some_extension_dir.getName())); } @Test @@ -306,28 +306,4 @@ public void testGetURLsForClasspath() throws Exception Assert.assertEquals(Sets.newHashSet(tmpDir2c.toURI().toURL(), tmpDir2d.toURI().toURL(), tmpDir2e.toURI().toURL()), Sets.newHashSet(urLsForClasspath.subList(3, 6))); } - - @Test - public void testExtensionsWithSameDirName() throws Exception - { - final String extensionName = "some_extension"; - final File tmpDir1 = temporaryFolder.newFolder(); - final File tmpDir2 = temporaryFolder.newFolder(); - final File extension1 = new File(tmpDir1, extensionName); - final File extension2 = new File(tmpDir2, extensionName); - Assert.assertTrue(extension1.mkdir()); - Assert.assertTrue(extension2.mkdir()); - final File jar1 = new File(extension1, "jar1.jar"); - final File jar2 = new File(extension2, "jar2.jar"); - - Assert.assertTrue(jar1.createNewFile()); - Assert.assertTrue(jar2.createNewFile()); - - final ExtensionsLoader extnLoader = new ExtensionsLoader(new ExtensionsConfig()); - final ClassLoader classLoader1 = extnLoader.getClassLoaderForExtension(extension1, false); - final ClassLoader classLoader2 = extnLoader.getClassLoaderForExtension(extension2, false); - - Assert.assertArrayEquals(new URL[]{jar1.toURI().toURL()}, ((URLClassLoader) classLoader1).getURLs()); - Assert.assertArrayEquals(new URL[]{jar2.toURI().toURL()}, ((URLClassLoader) classLoader2).getURLs()); - } } From a522e28e2846e072c86bff566f6197e1d7da2774 Mon Sep 17 00:00:00 2001 From: Paul Rogers Date: Thu, 26 Jan 2023 15:00:04 -0800 Subject: [PATCH 09/43] Extension tested in Broker server --- extensions-contrib/grpc-query/README.md | 60 ++- .../grpc/server/GrpcEndpointInitializer.java | 93 ++-- .../apache/druid/grpc/server/QueryDriver.java | 447 +++++++++++++++- .../druid/grpc/server/QueryDriverImpl.java | 479 ------------------ .../apache/druid/grpc/server/QueryServer.java | 4 +- .../druid/grpc/server/QueryService.java | 10 +- .../druid/grpc/server/ServerBuilder.java | 47 -- .../org/apache/druid/grpc/DriverTest.java | 6 +- .../org/apache/druid/grpc/GrpcQueryTest.java | 50 +- .../druid/grpc/GrpcQueryTestClient.java | 45 +- .../org/apache/druid/grpc/TestClient.java | 62 +++ .../apache/druid/grpc/server/QueryDriver.java | 28 - .../druid/grpc/server/QueryService.java | 48 -- .../druid/grpc/server/ServerBuilder.java | 46 -- 14 files changed, 628 insertions(+), 797 deletions(-) delete mode 100644 extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriverImpl.java delete mode 100644 extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/ServerBuilder.java create mode 100644 extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/TestClient.java delete mode 100644 extensions-contrib/grpc-shaded/src/main/java/org/apache/druid/grpc/server/QueryDriver.java delete mode 100644 extensions-contrib/grpc-shaded/src/main/java/org/apache/druid/grpc/server/QueryService.java delete mode 100644 extensions-contrib/grpc-shaded/src/main/java/org/apache/druid/grpc/server/ServerBuilder.java diff --git a/extensions-contrib/grpc-query/README.md b/extensions-contrib/grpc-query/README.md index 4916f81ead50..f6b76cb27624 100644 --- a/extensions-contrib/grpc-query/README.md +++ b/extensions-contrib/grpc-query/README.md @@ -139,12 +139,59 @@ This project contains several components: ## Debugging -Debugging of the gRPC extension requires extra care. +Debugging of the gRPC extension requires extra care due to the nuances of loading +classes from an extension. -### Debugging in an IDE +### Running in a Server -To debug the gRPC extension in a production-like environment, you'll want to debug the -extension in a running Broker. The easiest way to do this is: +Druid extensions are designed to run in the Druid server. The gRPC extension is +loaded only in the Druid broker using the contiguration described above. If something +fails during startup, the Broker will crash. Consult the Broker logs to determine +what went wrong. Startup failures are typically due to required jars not being installed +as part of the extension. Check the `pom.xml` file to track down what's missing. + +Failures can also occur when running a query. Such failures will result in a failure +response and should result in a log entry in the Broker log file. Use the log entry +to sort out what went wrong. + +You can also attach a debugger to the running process. You'll have to enable the debugger +in the server by adding the required parameters to the Broker's `jvm.config` file. + +### Debugging using Unit Tests + +To debug the functionality of the extension, your best bet is to debug in the context +of a unit test. Druid provides a special test-only SQL stack with a few pre-defined +datasources. See the various `CalciteQueryTest` classes to see what these are. You can +also query Druid's various system tables. See `GrpcQueryTest` for a simple "starter" +unit test that configures the server and uses an in-process client to send requests. + +Most unit testing can be done without the gRPC server, by calling the `QueryDriver` +class directly. That is, if the goal is work with the code that takes a request, runs +a query, and produces a response, then the driver is the key and the server is just a +bit of extra copmlexity. See the `DriverTest` class for an example unit test. + +### Debugging in a Server in an IDE + +We would like to be able to debug the gRPC extension, within the Broker, in an IDE. +As it turns out, doing so breaks Druid's class loader mechanisms in ways that are both +hard to understand and hard to work around. When run in a server, Java creates an instance +of `GrpcQueryModule` using the extension's class loader. Java then uses that same class +loader to load other classes in the extension, including those here and those in the +shaded gRPC jar file. + +However, when run in an IDE, if this project is on the class path, then the `GrpcQueryModule` +class will be loaded from the "App" class loader. This works fine: it causes the other +classes of this module to also be loaded from the class path. However, once execution +calls into gRPC, Java will use the App class loader, not the extension class loader, and +will fail to find some of the classes, resulting in Java exceptions. Worse, in some cases, +Java may load the same class from both class loaders. To Java, these are not the same +classes, and you will get mysterious errors as a result. + +For now, the lesson is: don't try to debug the extension in the Broker in the IDE. Use +one of the above options instead. + +For reference (and in case we figure out a solution to the class loader conflict), +the way to debug the Broker in an IDE is the following: * Build your branch. Use the `-P bundle-contrib-exts` flag in place of `-P dist`, as described above. @@ -156,7 +203,7 @@ extension in a running Broker. The easiest way to do this is: * In your IDE, define a launch configuration for the Broker. * The launch command is `server broker` * Add the following JVM arguments: - + ```text --add-exports java.base/jdk.internal.perf=ALL-UNNAMED --add-exports jdk.management/com.sun.management.internal=ALL-UNNAMED @@ -201,3 +248,6 @@ This is not the first project to have created a gRPC API for Druid. Others inclu * [Druid gRPC-json server extension](https://github.com/apache/druid/pull/6798) Full credit goes to those who have gone this way before. + +Note that the class loader solution used by the two code bases above turned out +to not be needed. See the notes above about the class loader issues. diff --git a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/GrpcEndpointInitializer.java b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/GrpcEndpointInitializer.java index 6583185620f8..27a44034aa2a 100644 --- a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/GrpcEndpointInitializer.java +++ b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/GrpcEndpointInitializer.java @@ -20,7 +20,6 @@ package org.apache.druid.grpc.server; import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.guice.ExtensionsLoader; import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.guice.annotations.Json; import org.apache.druid.guice.annotations.NativeQuery; @@ -32,22 +31,24 @@ import javax.inject.Inject; +import java.io.IOException; + +/** + * Initializes the gRPC endpoint (server). This version uses a Netty-based server + * separate from Druid's primary Jetty-based server. We may want to consider a + * + * recent addition to the gRPC examples to run gRPC as a servlet. + *

+ * An instance of this class is created by Guice and managed via Druid's + * livecycle manager. + */ @ManageLifecycle public class GrpcEndpointInitializer { private static final Logger log = new Logger(GrpcEndpointInitializer.class); - /** - * Name of the extension. Must match the name in the - * {@code $DRUID_HOME/extensions} directory. - */ - public static final String EXTENSION_NAME = "grpc-query"; - private final GrpcQueryConfig config; -// private final QueryDriver driver; - private final ClassLoader extnClassLoader; - private final ObjectMapper jsonMapper; - private final SqlStatementFactory sqlStatementFactory; + private final QueryDriver driver; private QueryServer server; @@ -55,63 +56,37 @@ public class GrpcEndpointInitializer public GrpcEndpointInitializer( GrpcQueryConfig config, final @Json ObjectMapper jsonMapper, - final @NativeQuery SqlStatementFactory sqlStatementFactory, - final ExtensionsLoader extensionsLoader + final @NativeQuery SqlStatementFactory sqlStatementFactory ) { this.config = config; -// this.driver = new QueryDriver(jsonMapper, sqlStatementFactory); - this.jsonMapper = jsonMapper; - this.sqlStatementFactory = sqlStatementFactory; - - // Retrieve the class loader for this extension. Necessary because, in an IDE, - // the class loader for the extension's files will be the AppClassLoader since - // the IDE puts our classes on the class path. - this.extnClassLoader = extensionsLoader.getClassLoaderForExtension(EXTENSION_NAME); - if (this.extnClassLoader == null) { - throw new ISE("No extension class loader for %s: wrong name?", EXTENSION_NAME); - } + this.driver = new QueryDriver(jsonMapper, sqlStatementFactory); } @LifecycleStart public void start() { -// ClassLoader thisLoader = getClass().getClassLoader(); - final ClassLoader oldLoader = Thread.currentThread().getContextClassLoader(); - + server = new QueryServer(config.getPort(), driver); try { - Thread.currentThread().setContextClassLoader(extnClassLoader); -// try { -// Class found = extnClassLoader.loadClass("io.netty.handler.codec.http2.Http2Headers"); -// } catch (ClassNotFoundException e) { -// throw new ISE(e, "Fatal error: grpc query server startup failed"); -// } + server.start(); + } + catch (IOException e) { + // Indicates an error when gRPC tried to start the server + // (such the port is already in use.) + log.error(e, "Fatal error: gRPC query server startup failed"); - try { - QueryDriverImpl driver = new QueryDriverImpl(jsonMapper, sqlStatementFactory); - Class serverClass = extnClassLoader.loadClass("org.apache.druid.grpc.server.QueryServer"); - Class driverClass = extnClassLoader.loadClass("org.apache.druid.grpc.server.QueryDriver"); -// Object driver = driverClass -// .getConstructor(ObjectMapper.class, SqlStatementFactory.class) -// .newInstance(jsonMapper, sqlStatementFactory); -// Class builderClass = extnClassLoader.loadClass("org.apache.druid.grpc.server.ServerBuilder"); -// ServerBuilder builder = builderClass.getConstructor().newInstance(); -// server = builder.buildServer(config.getPort(), driver); - server = (QueryServer) serverClass - .getConstructor(Integer.class, driverClass) - .newInstance(config.getPort(), driver); + // This exception will bring down the Broker as there is not much we can + // do if we can't start the gRPC endpoint. + throw new ISE(e, "Fatal error: grpc query server startup failed"); + } + catch (Throwable t) { + // Catch-all for other errors. The most likely error is that some class was not found + // (that is, class loader issues in an IDE, or a jar missing in the extension). + log.error(t, "Fatal error: gRPC query server startup failed"); -// server = new QueryServer(config.getPort(), driver); - server.start(); - } catch (Exception e) { - log.error(e, "Fatal error: gRPC query server startup failed"); - throw new ISE(e, "Fatal error: grpc query server startup failed"); - } catch (Throwable t) { - log.error(t, "Fatal error: gRPC query server startup failed"); - throw t; - } - } finally { - Thread.currentThread().setContextClassLoader(oldLoader); + // This exception will bring down the Broker as there is not much we can + // do if we can't start the gRPC endpoint. + throw t; } } @@ -121,7 +96,9 @@ public void stop() if (server != null) { try { server.blockUntilShutdown(); - } catch (InterruptedException e) { + } + catch (InterruptedException e) { + // Just warn. We're shutting down anyway, so no need to throw an exception. log.warn(e, "gRPC query server shutdown failed"); } server = null; diff --git a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriver.java b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriver.java index 56e7b8718981..74591edf1645 100644 --- a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriver.java +++ b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriver.java @@ -19,10 +19,453 @@ package org.apache.druid.grpc.server; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableMap; +import com.google.protobuf.ByteString; +import org.apache.calcite.avatica.SqlType; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.druid.grpc.proto.QueryOuterClass.ColumnSchema; +import org.apache.druid.grpc.proto.QueryOuterClass.DruidType; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryParameter; import org.apache.druid.grpc.proto.QueryOuterClass.QueryRequest; import org.apache.druid.grpc.proto.QueryOuterClass.QueryResponse; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryResultFormat; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryStatus; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.guava.Accumulator; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.server.security.Access; +import org.apache.druid.server.security.AuthenticationResult; +import org.apache.druid.server.security.ForbiddenException; +import org.apache.druid.sql.DirectStatement; +import org.apache.druid.sql.DirectStatement.ResultSet; +import org.apache.druid.sql.SqlPlanningException; +import org.apache.druid.sql.SqlQueryPlus; +import org.apache.druid.sql.SqlRowTransformer; +import org.apache.druid.sql.SqlStatementFactory; +import org.apache.druid.sql.calcite.table.RowSignatures; +import org.apache.druid.sql.http.ResultFormat; +import org.apache.druid.sql.http.SqlParameter; -public interface QueryDriver +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +/** + * "Driver" for the gRPC query endpoint. Handles translating the gRPC {@link QueryRequest} + * into Druid's internal formats, running the query, and translating the results into a + * gRPC {@link QueryResponse}. Allows for easier unit testing as we separate the machinery + * of running a query, given the request, from the gRPC server machinery. + */ +public class QueryDriver { - QueryResponse submitQuery(QueryRequest request); + /** + * Internal runtime exception to report request errors. + */ + private static class RequestError extends RuntimeException + { + public RequestError(String msg) + { + super(msg); + } + } + + private final ObjectMapper jsonMapper; + private final SqlStatementFactory sqlStatementFactory; + + public QueryDriver( + final ObjectMapper jsonMapper, + final SqlStatementFactory sqlStatementFactory + ) + { + this.jsonMapper = Preconditions.checkNotNull(jsonMapper, "jsonMapper"); + this.sqlStatementFactory = Preconditions.checkNotNull(sqlStatementFactory, "sqlStatementFactory"); + } + + /** + * First-cut synchronous query handler. Druid prefers to stream results, in + * part to avoid overly-short network timeouts. However, for now, we simply run + * the query within this call and prepare the Protobuf response. Async handling + * can come later. + */ + public QueryResponse submitQuery(QueryRequest request, AuthenticationResult authResult) + { + final SqlQueryPlus queryPlus; + try { + queryPlus = translateQuery(request, authResult); + } + catch (RuntimeException e) { + return QueryResponse.newBuilder() + .setQueryId("") + .setStatus(QueryStatus.REQUEST_ERROR) + .setErrorMessage(e.getMessage()) + .build(); + } + final DirectStatement stmt = sqlStatementFactory.directStatement(queryPlus); + final String currThreadName = Thread.currentThread().getName(); + try { + Thread.currentThread().setName(StringUtils.format("grpc-sql[%s]", stmt.sqlQueryId())); + final ResultSet thePlan = stmt.plan(); + final SqlRowTransformer rowTransformer = thePlan.createRowTransformer(); + final ByteString results = encodeResults(request.getResultFormat(), thePlan, rowTransformer); + stmt.reporter().succeeded(0); // TODO: real byte count (of payload) + stmt.close(); + return QueryResponse.newBuilder() + .setQueryId(stmt.sqlQueryId()) + .setStatus(QueryStatus.OK) + .setData(results) + .clearErrorMessage() + .addAllColumns(encodeColumns(rowTransformer)) + .build(); + } + catch (ForbiddenException e) { + stmt.reporter().failed(e); + stmt.close(); + return QueryResponse.newBuilder() + .setQueryId(stmt.sqlQueryId()) + .setStatus(QueryStatus.UNAUTHORIZED) + .setErrorMessage(Access.DEFAULT_ERROR_MESSAGE) + .build(); + } + catch (RequestError e) { + stmt.reporter().failed(e); + stmt.close(); + return QueryResponse.newBuilder() + .setQueryId(stmt.sqlQueryId()) + .setStatus(QueryStatus.REQUEST_ERROR) + .setErrorMessage(e.getMessage()) + .build(); + } + catch (SqlPlanningException e) { + stmt.reporter().failed(e); + stmt.close(); + return QueryResponse.newBuilder() + .setQueryId(stmt.sqlQueryId()) + .setStatus(QueryStatus.INVALID_SQL) + .setErrorMessage(e.getMessage()) + .build(); + } + catch (IOException | RuntimeException e) { + stmt.reporter().failed(e); + stmt.close(); + return QueryResponse.newBuilder() + .setQueryId(stmt.sqlQueryId()) + .setStatus(QueryStatus.RUNTIME_ERROR) + .setErrorMessage(e.getMessage()) + .build(); + } + // There is a claim that Calcite sometimes throws a java.lang.AssertionError, but we do not have a test that can + // reproduce it checked into the code (the best we have is something that uses mocks to throw an Error, which is + // dubious at best). We keep this just in case, but it might be best to remove it and see where the + // AssertionErrors are coming from and do something to ensure that they don't actually make it out of Calcite + catch (AssertionError e) { + stmt.reporter().failed(e); + stmt.close(); + return QueryResponse.newBuilder() + .setQueryId(stmt.sqlQueryId()) + .setStatus(QueryStatus.RUNTIME_ERROR) + .setErrorMessage(e.getMessage()) + .build(); + } + finally { + Thread.currentThread().setName(currThreadName); + } + } + + /** + * Convert the rRPC query format to the internal {@link SqlQueryPlus} format. + */ + private SqlQueryPlus translateQuery(QueryRequest request, AuthenticationResult authResult) + { + return SqlQueryPlus.builder() + .sql(request.getQuery()) + .context(translateContext(request)) + .sqlParameters(translateParameters(request)) + .auth(authResult) + .build(); + } + + /** + * Translate the query context from the gRPC format to the internal format. When + * read from REST/JSON, the JSON translator will convert the type of each value + * into a number, Boolean, etc. gRPC has no similar feature. Rather than clutter up + * the gRPC request with typed context values, we rely on the existing code that can + * translate string values to the desired type on the fly. Thus, we build up a + * {@code Map}. + */ + private Map translateContext(QueryRequest request) + { + ImmutableMap.Builder builder = ImmutableMap.builder(); + if (request.getContextCount() > 0) { + for (Map.Entry entry : request.getContextMap().entrySet()) { + builder.put(entry.getKey(), entry.getValue()); + } + } + return builder.build(); + } + + /** + * Convert the gRPC parameter format to the internal Druid {@link SqlParameter} + * format. That format is then again translated by the {@link SqlQueryPlus} class. + */ + private List translateParameters(QueryRequest request) + { + if (request.getParametersCount() == 0) { + return null; + } + List params = new ArrayList<>(); + for (QueryParameter value : request.getParametersList()) { + params.add(translateParameter(value)); + } + return params; + } + + private SqlParameter translateParameter(QueryParameter value) + { + switch (value.getValueCase()) { + case ARRAYVALUE: + // Not yet supported: waiting for an open PR + return null; + case DOUBLEVALUE: + return new SqlParameter(SqlType.DOUBLE, value.getDoubleValue()); + case LONGVALUE: + return new SqlParameter(SqlType.BIGINT, value.getLongValue()); + case STRINGVALUE: + return new SqlParameter(SqlType.VARCHAR, value.getStringValue()); + case NULLVALUE: + case VALUE_NOT_SET: + return null; + default: + throw new RequestError("Invalid parameter type: " + value.getValueCase().name()); + } + } + + /** + * Translate the column schema from the Druid internal form to the gRPC + * {@link ColumnSchema} form. Note that since the gRPC response returns the + * schema, none of the data formats include a header. This makes the data format + * simpler and cleaner. + */ + private Iterable encodeColumns(SqlRowTransformer rowTransformer) + { + RelDataType rowType = rowTransformer.getRowType(); + final RowSignature signature = RowSignatures.fromRelDataType(rowType.getFieldNames(), rowType); + List cols = new ArrayList<>(); + for (int i = 0; i < rowType.getFieldCount(); i++) { + ColumnSchema col = ColumnSchema.newBuilder() + .setName(signature.getColumnName(i)) + .setSqlType(rowType.getFieldList().get(i).getType().getSqlTypeName().getName()) + .setDruidType(convertDruidType(signature.getColumnType(i))) + .build(); + cols.add(col); + } + return cols; + } + + /** + * Convert from Druid's internal format of the Druid data type to the gRPC form. + */ + private DruidType convertDruidType(Optional colType) + { + if (!colType.isPresent()) { + return DruidType.UNKNOWN_TYPE; + } + ColumnType druidType = colType.get(); + if (druidType == ColumnType.STRING) { + return DruidType.STRING; + } + if (druidType == ColumnType.STRING_ARRAY) { + return DruidType.STRING_ARRAY; + } + if (druidType == ColumnType.LONG) { + return DruidType.LONG; + } + if (druidType == ColumnType.LONG_ARRAY) { + return DruidType.LONG_ARRAY; + } + if (druidType == ColumnType.FLOAT) { + return DruidType.FLOAT; + } + if (druidType == ColumnType.FLOAT_ARRAY) { + return DruidType.FLOAT_ARRAY; + } + if (druidType == ColumnType.DOUBLE) { + return DruidType.DOUBLE; + } + if (druidType == ColumnType.DOUBLE_ARRAY) { + return DruidType.DOUBLE_ARRAY; + } + if (druidType == ColumnType.UNKNOWN_COMPLEX) { + return DruidType.COMPLEX; + } + return DruidType.UNKNOWN_TYPE; + } + + /** + * Generic mechanism to write query results to one of the supported gRPC formats. + */ + public interface GrpcResultWriter + { + void start() throws IOException; + void writeRow(Object[] row) throws IOException; + void close() throws IOException; + } + + /** + * Writer for the SQL result formats. Reuses the SQL format writer implementations. + * Note: gRPC does not use the headers: schema information is available in the + * rRPC response. + */ + public static class GrpcResultFormatWriter implements GrpcResultWriter + { + protected final ResultFormat.Writer formatWriter; + protected final SqlRowTransformer rowTransformer; + + public GrpcResultFormatWriter( + final ResultFormat.Writer formatWriter, + final SqlRowTransformer rowTransformer + ) + { + this.formatWriter = formatWriter; + this.rowTransformer = rowTransformer; + } + + @Override + public void start() throws IOException + { + } + + @Override + public void writeRow(Object[] row) throws IOException + { + formatWriter.writeRowStart(); + for (int i = 0; i < rowTransformer.getFieldList().size(); i++) { + final Object value = rowTransformer.transform(row, i); + formatWriter.writeRowField(rowTransformer.getFieldList().get(i), value); + } + formatWriter.writeRowEnd(); + } + + @Override + public void close() throws IOException + { + formatWriter.close(); + } + } + + /** + * Internal runtime exception to pass {@link IOException}s though the + * {@link Sequence} {@link Accumulator} protocol. + */ + private static class ResponseError extends RuntimeException + { + public ResponseError(IOException e) + { + super(e); + } + } + + /** + * Druid query results use a complex {@link Sequence} mechanism. This class uses an + * {@link Accumulator} to walk the results and present each to the associated + * results writer. This is a very rough analogy of the {@code SqlResourceQueryResultPusher} + * in the REST {@code SqlResource} class. + */ + public static class GrpcResultsAccumulator implements Accumulator + { + private final GrpcResultWriter writer; + + public GrpcResultsAccumulator(final GrpcResultWriter writer) + { + this.writer = writer; + } + + public void push(org.apache.druid.server.QueryResponse queryResponse) throws IOException + { + final Sequence results = queryResponse.getResults(); + writer.start(); + try { + results.accumulate(null, this); + } + catch (ResponseError e) { + throw (IOException) e.getCause(); + } + writer.close(); + } + + @Override + public Void accumulate(Void accumulated, Object[] in) + { + try { + writer.writeRow(in); + } + catch (IOException e) { + throw new ResponseError(e); + } + return null; + } + } + + /** + * Convert the query results to a set of bytes to be attached to the query response. + *

+ * This version is pretty basic: the results are materialized as a byte array. That's + * fine for small result sets, but should be rethought for larger result sets. + */ + private ByteString encodeResults( + final QueryResultFormat queryResultFormat, + final ResultSet thePlan, + final SqlRowTransformer rowTransformer + ) throws IOException + { + // Accumulate the results as a byte array. + ByteArrayOutputStream out = new ByteArrayOutputStream(); + GrpcResultWriter writer; + + // For the SQL-supported formats, use the SQL-provided writers. + switch (queryResultFormat) { + case CSV: + writer = new GrpcResultFormatWriter( + ResultFormat.CSV.createFormatter(out, jsonMapper), + rowTransformer + ); + break; + case JSON_ARRAY: + writer = new GrpcResultFormatWriter( + ResultFormat.ARRAY.createFormatter(out, jsonMapper), + rowTransformer + ); + break; + case JSON_ARRAY_LINES: + writer = new GrpcResultFormatWriter( + ResultFormat.ARRAYLINES.createFormatter(out, jsonMapper), + rowTransformer + ); + break; + + // TODO: Provide additional writers for the other formats which we + // want in gRPC. + case JSON_OBJECT: + throw new UnsupportedOperationException(); // TODO + case JSON_OBJECT_LINES: + throw new UnsupportedOperationException(); // TODO + case PROTOBUF_INLINE: + throw new UnsupportedOperationException(); // TODO + + // This is the hard one: encode the results as a Protobuf array. + case PROTOBUF_RESPONSE: + throw new UnsupportedOperationException(); // TODO + default: + throw new RequestError("Unsupported query result format"); + } + GrpcResultsAccumulator accumulator = new GrpcResultsAccumulator(writer); + accumulator.push(thePlan.run()); + return ByteString.copyFrom(out.toByteArray()); + } } diff --git a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriverImpl.java b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriverImpl.java deleted file mode 100644 index 79cbb7875440..000000000000 --- a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriverImpl.java +++ /dev/null @@ -1,479 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.grpc.server; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableMap; -import com.google.protobuf.ByteString; -import org.apache.calcite.avatica.SqlType; -import org.apache.calcite.rel.type.RelDataType; -import org.apache.druid.grpc.proto.QueryOuterClass.ColumnSchema; -import org.apache.druid.grpc.proto.QueryOuterClass.DruidType; -import org.apache.druid.grpc.proto.QueryOuterClass.QueryParameter; -import org.apache.druid.grpc.proto.QueryOuterClass.QueryRequest; -import org.apache.druid.grpc.proto.QueryOuterClass.QueryResponse; -import org.apache.druid.grpc.proto.QueryOuterClass.QueryResultFormat; -import org.apache.druid.grpc.proto.QueryOuterClass.QueryStatus; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.guava.Accumulator; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.server.security.Access; -import org.apache.druid.server.security.AuthConfig; -import org.apache.druid.server.security.AuthenticationResult; -import org.apache.druid.server.security.ForbiddenException; -import org.apache.druid.sql.DirectStatement; -import org.apache.druid.sql.DirectStatement.ResultSet; -import org.apache.druid.sql.SqlPlanningException; -import org.apache.druid.sql.SqlQueryPlus; -import org.apache.druid.sql.SqlRowTransformer; -import org.apache.druid.sql.SqlStatementFactory; -import org.apache.druid.sql.calcite.table.RowSignatures; -import org.apache.druid.sql.http.ResultFormat; -import org.apache.druid.sql.http.SqlParameter; - -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.Optional; - -/** - * "Driver" for the gRPC query endpoint. Handles translating the gRPC {@link QueryRequest} - * into Druid's internal formats, running the query, and translating the results into a - * gRPC {@link QueryResponse}. Allows for easier unit testing as we separate the machinery - * of running a query, given the request, from the gRPC server machinery. - */ -public class QueryDriverImpl implements QueryDriver -{ - /** - * Internal runtime exception to report request errors. - */ - private static class RequestError extends RuntimeException - { - public RequestError(String msg) - { - super(msg); - } - } - - private final ObjectMapper jsonMapper; - private final SqlStatementFactory sqlStatementFactory; - - public QueryDriverImpl( - final ObjectMapper jsonMapper, - final SqlStatementFactory sqlStatementFactory - ) - { - this.jsonMapper = Preconditions.checkNotNull(jsonMapper, "jsonMapper"); - this.sqlStatementFactory = Preconditions.checkNotNull(sqlStatementFactory, "sqlStatementFactory"); - } - - /** - * First-cut synchronous query handler. Druid prefers to stream results, in - * part to avoid overly-short network timeouts. However, for now, we simply run - * the query within this call and prepare the Protobuf response. Async handling - * can come later. - */ - @Override - public QueryResponse submitQuery(QueryRequest request) - { - // TODO: How will we get the auth result for gRPC? - AuthenticationResult authResult = new AuthenticationResult( - "superUser", - AuthConfig.ALLOW_ALL_NAME, - null, null - ); - final SqlQueryPlus queryPlus; - try { - queryPlus = translateQuery(request, authResult); - } - catch (RuntimeException e) { - return QueryResponse.newBuilder() - .setQueryId("") - .setStatus(QueryStatus.REQUEST_ERROR) - .setErrorMessage(e.getMessage()) - .build(); - } - final DirectStatement stmt = sqlStatementFactory.directStatement(queryPlus); - final String currThreadName = Thread.currentThread().getName(); - try { - Thread.currentThread().setName(StringUtils.format("grpc-sql[%s]", stmt.sqlQueryId())); - final ResultSet thePlan = stmt.plan(); - final SqlRowTransformer rowTransformer = thePlan.createRowTransformer(); - final ByteString results = encodeResults(request.getResultFormat(), thePlan, rowTransformer); - stmt.reporter().succeeded(0); // TODO: real byte count (of payload) - stmt.close(); - return QueryResponse.newBuilder() - .setQueryId(stmt.sqlQueryId()) - .setStatus(QueryStatus.OK) - .setData(results) - .clearErrorMessage() - .addAllColumns(encodeColumns(rowTransformer)) - .build(); - } - catch (ForbiddenException e) { - stmt.reporter().failed(e); - stmt.close(); - return QueryResponse.newBuilder() - .setQueryId(stmt.sqlQueryId()) - .setStatus(QueryStatus.UNAUTHORIZED) - .setErrorMessage(Access.DEFAULT_ERROR_MESSAGE) - .build(); - } - catch (RequestError e) { - stmt.reporter().failed(e); - stmt.close(); - return QueryResponse.newBuilder() - .setQueryId(stmt.sqlQueryId()) - .setStatus(QueryStatus.REQUEST_ERROR) - .setErrorMessage(e.getMessage()) - .build(); - } - catch (SqlPlanningException e) { - stmt.reporter().failed(e); - stmt.close(); - return QueryResponse.newBuilder() - .setQueryId(stmt.sqlQueryId()) - .setStatus(QueryStatus.INVALID_SQL) - .setErrorMessage(e.getMessage()) - .build(); - } - catch (IOException | RuntimeException e) { - stmt.reporter().failed(e); - stmt.close(); - return QueryResponse.newBuilder() - .setQueryId(stmt.sqlQueryId()) - .setStatus(QueryStatus.RUNTIME_ERROR) - .setErrorMessage(e.getMessage()) - .build(); - } - // There is a claim that Calcite sometimes throws a java.lang.AssertionError, but we do not have a test that can - // reproduce it checked into the code (the best we have is something that uses mocks to throw an Error, which is - // dubious at best). We keep this just in case, but it might be best to remove it and see where the - // AssertionErrors are coming from and do something to ensure that they don't actually make it out of Calcite - catch (AssertionError e) { - stmt.reporter().failed(e); - stmt.close(); - return QueryResponse.newBuilder() - .setQueryId(stmt.sqlQueryId()) - .setStatus(QueryStatus.RUNTIME_ERROR) - .setErrorMessage(e.getMessage()) - .build(); - } - finally { - Thread.currentThread().setName(currThreadName); - } - } - - /** - * Convert the rRPC query format to the internal {@link SqlQueryPlus} format. - */ - private SqlQueryPlus translateQuery(QueryRequest request, AuthenticationResult authResult) - { - return SqlQueryPlus.builder() - .sql(request.getQuery()) - .context(translateContext(request)) - .sqlParameters(translateParameters(request)) - .auth(authResult) - .build(); - } - - /** - * Translate the query context from the gRPC format to the internal format. When - * read from REST/JSON, the JSON translator will convert the type of each value - * into a number, Boolean, etc. gRPC has no similar feature. Rather than clutter up - * the gRPC request with typed context values, we rely on the existing code that can - * translate string values to the desired type on the fly. Thus, we build up a - * {@code Map}. - */ - private Map translateContext(QueryRequest request) - { - ImmutableMap.Builder builder = ImmutableMap.builder(); - if (request.getContextCount() > 0) { - for (Map.Entry entry : request.getContextMap().entrySet()) { - builder.put(entry.getKey(), entry.getValue()); - } - } - return builder.build(); - } - - /** - * Convert the gRPC parameter format to the internal Druid {@link SqlParameter} - * format. That format is then again translated by the {@link SqlQueryPlus} class. - */ - private List translateParameters(QueryRequest request) - { - if (request.getParametersCount() == 0) { - return null; - } - List params = new ArrayList<>(); - for (QueryParameter value : request.getParametersList()) { - params.add(translateParameter(value)); - } - return params; - } - - private SqlParameter translateParameter(QueryParameter value) - { - switch (value.getValueCase()) { - case ARRAYVALUE: - // Not yet supported: waiting for an open PR - return null; - case DOUBLEVALUE: - return new SqlParameter(SqlType.DOUBLE, value.getDoubleValue()); - case LONGVALUE: - return new SqlParameter(SqlType.BIGINT, value.getLongValue()); - case STRINGVALUE: - return new SqlParameter(SqlType.VARCHAR, value.getStringValue()); - case NULLVALUE: - case VALUE_NOT_SET: - return null; - default: - throw new RequestError("Invalid parameter type: " + value.getValueCase().name()); - } - } - - /** - * Translate the column schema from the Druid internal form to the gRPC - * {@link ColumnSchema} form. Note that since the gRPC response returns the - * schema, none of the data formats include a header. This makes the data format - * simpler and cleaner. - */ - private Iterable encodeColumns(SqlRowTransformer rowTransformer) - { - RelDataType rowType = rowTransformer.getRowType(); - final RowSignature signature = RowSignatures.fromRelDataType(rowType.getFieldNames(), rowType); - List cols = new ArrayList<>(); - for (int i = 0; i < rowType.getFieldCount(); i++) { - ColumnSchema col = ColumnSchema.newBuilder() - .setName(signature.getColumnName(i)) - .setSqlType(rowType.getFieldList().get(i).getType().getSqlTypeName().getName()) - .setDruidType(convertDruidType(signature.getColumnType(i))) - .build(); - cols.add(col); - } - return cols; - } - - /** - * Convert from Druid's internal format of the Druid data type to the gRPC form. - */ - private DruidType convertDruidType(Optional colType) - { - if (!colType.isPresent()) { - return DruidType.UNKNOWN_TYPE; - } - ColumnType druidType = colType.get(); - if (druidType == ColumnType.STRING) { - return DruidType.STRING; - } - if (druidType == ColumnType.STRING_ARRAY) { - return DruidType.STRING_ARRAY; - } - if (druidType == ColumnType.LONG) { - return DruidType.LONG; - } - if (druidType == ColumnType.LONG_ARRAY) { - return DruidType.LONG_ARRAY; - } - if (druidType == ColumnType.FLOAT) { - return DruidType.FLOAT; - } - if (druidType == ColumnType.FLOAT_ARRAY) { - return DruidType.FLOAT_ARRAY; - } - if (druidType == ColumnType.DOUBLE) { - return DruidType.DOUBLE; - } - if (druidType == ColumnType.DOUBLE_ARRAY) { - return DruidType.DOUBLE_ARRAY; - } - if (druidType == ColumnType.UNKNOWN_COMPLEX) { - return DruidType.COMPLEX; - } - return DruidType.UNKNOWN_TYPE; - } - - /** - * Generic mechanism to write query results to one of the supported gRPC formats. - */ - public interface GrpcResultWriter - { - void start() throws IOException; - void writeRow(Object[] row) throws IOException; - void close() throws IOException; - } - - /** - * Writer for the SQL result formats. Reuses the SQL format writer implementations. - * Note: gRPC does not use the headers: schema information is available in the - * rRPC response. - */ - public static class GrpcResultFormatWriter implements GrpcResultWriter - { - protected final ResultFormat.Writer formatWriter; - protected final SqlRowTransformer rowTransformer; - - public GrpcResultFormatWriter( - final ResultFormat.Writer formatWriter, - final SqlRowTransformer rowTransformer - ) - { - this.formatWriter = formatWriter; - this.rowTransformer = rowTransformer; - } - - @Override - public void start() throws IOException - { - } - - @Override - public void writeRow(Object[] row) throws IOException - { - formatWriter.writeRowStart(); - for (int i = 0; i < rowTransformer.getFieldList().size(); i++) { - final Object value = rowTransformer.transform(row, i); - formatWriter.writeRowField(rowTransformer.getFieldList().get(i), value); - } - formatWriter.writeRowEnd(); - } - - @Override - public void close() throws IOException - { - formatWriter.close(); - } - } - - /** - * Internal runtime exception to pass {@link IOException}s though the - * {@link Sequence} {@link Accumulator} protocol. - */ - private static class ResponseError extends RuntimeException - { - public ResponseError(IOException e) - { - super(e); - } - } - - /** - * Druid query results use a complex {@link Sequence} mechanism. This class uses an - * {@link Accumulator} to walk the results and present each to the associated - * results writer. This is a very rough analogy of the {@code SqlResourceQueryResultPusher} - * in the REST {@code SqlResource} class. - */ - public static class GrpcResultsAccumulator implements Accumulator - { - private final GrpcResultWriter writer; - - public GrpcResultsAccumulator(final GrpcResultWriter writer) - { - this.writer = writer; - } - - public void push(org.apache.druid.server.QueryResponse queryResponse) throws IOException - { - final Sequence results = queryResponse.getResults(); - writer.start(); - try { - results.accumulate(null, this); - } - catch (ResponseError e) { - throw (IOException) e.getCause(); - } - writer.close(); - } - - @Override - public Void accumulate(Void accumulated, Object[] in) - { - try { - writer.writeRow(in); - } - catch (IOException e) { - throw new ResponseError(e); - } - return null; - } - } - - /** - * Convert the query results to a set of bytes to be attached to the query response. - *

- * This version is pretty basic: the results are materialized as a byte array. That's - * fine for small result sets, but should be rethought for larger result sets. - */ - private ByteString encodeResults( - final QueryResultFormat queryResultFormat, - final ResultSet thePlan, - final SqlRowTransformer rowTransformer - ) throws IOException - { - // Accumulate the results as a byte array. - ByteArrayOutputStream out = new ByteArrayOutputStream(); - GrpcResultWriter writer; - - // For the SQL-supported formats, use the SQL-provided writers. - switch (queryResultFormat) { - case CSV: - writer = new GrpcResultFormatWriter( - ResultFormat.CSV.createFormatter(out, jsonMapper), - rowTransformer - ); - break; - case JSON_ARRAY: - writer = new GrpcResultFormatWriter( - ResultFormat.ARRAY.createFormatter(out, jsonMapper), - rowTransformer - ); - break; - case JSON_ARRAY_LINES: - writer = new GrpcResultFormatWriter( - ResultFormat.ARRAYLINES.createFormatter(out, jsonMapper), - rowTransformer - ); - break; - - // TODO: Provide additional writers for the other formats which we - // want in gRPC. - case JSON_OBJECT: - throw new UnsupportedOperationException(); // TODO - case JSON_OBJECT_LINES: - throw new UnsupportedOperationException(); // TODO - case PROTOBUF_INLINE: - throw new UnsupportedOperationException(); // TODO - - // This is the hard one: encode the results as a Protobuf array. - case PROTOBUF_RESPONSE: - throw new UnsupportedOperationException(); // TODO - default: - throw new RequestError("Unsupported query result format"); - } - GrpcResultsAccumulator accumulator = new GrpcResultsAccumulator(writer); - accumulator.push(thePlan.run()); - return ByteString.copyFrom(out.toByteArray()); - } -} diff --git a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryServer.java b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryServer.java index c8784b254f4b..b648ceeabcd1 100644 --- a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryServer.java +++ b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryServer.java @@ -32,7 +32,7 @@ /** * Basic gRPC server adapted from the gRPC examples. Delegates to the - * {@link QueryDriverImpl} class to do the actual work of running the query. + * {@link QueryDriver} class to do the actual work of running the query. *

* This class is preliminary. It is good enough for unit tests, but a bit more work * is needed to integrate this class into the Druid server. @@ -59,7 +59,7 @@ public QueryServer( public void start() throws IOException { - server = Grpc.newServerBuilderForPort(port, InsecureServerCredentials.create()) + server = Grpc.newServerBuilderForPort(port, InsecureServerCredentials.create()) .addService(new QueryService(driver)) .build() .start(); diff --git a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryService.java b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryService.java index 19e2cf1b18a4..54ae92d8d3f5 100644 --- a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryService.java +++ b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryService.java @@ -23,6 +23,8 @@ import org.apache.druid.grpc.proto.QueryGrpc; import org.apache.druid.grpc.proto.QueryOuterClass.QueryRequest; import org.apache.druid.grpc.proto.QueryOuterClass.QueryResponse; +import org.apache.druid.server.security.AuthConfig; +import org.apache.druid.server.security.AuthenticationResult; /** * Implementation of the gRPC Query service. Provides a single method @@ -41,7 +43,13 @@ public QueryService(QueryDriver driver) @Override public void submitQuery(QueryRequest request, StreamObserver responseObserver) { - QueryResponse reply = driver.submitQuery(request); + // TODO: How will we get the auth result for gRPC? + AuthenticationResult authResult = new AuthenticationResult( + "superUser", + AuthConfig.ALLOW_ALL_NAME, + null, null + ); + QueryResponse reply = driver.submitQuery(request, authResult); responseObserver.onNext(reply); responseObserver.onCompleted(); } diff --git a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/ServerBuilder.java b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/ServerBuilder.java deleted file mode 100644 index 87dd878e1437..000000000000 --- a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/ServerBuilder.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.grpc.server; - -import io.grpc.BindableService; -import io.grpc.Grpc; -import io.grpc.InsecureServerCredentials; -import io.grpc.Server; - -/** - * Builds the Grpc Server given a port. This builds a Netty-based server - * internally. Done in the shaded jar so that this class can be created - * using the extension class loader, so that the gRPC classes are also - * created using that class loader which allows gRPC to find its components. - * The rest of the server is created implicitly which will use the - * the extension class loader in production, the app loader when debugging. - * If you find you are getting errors about classes not found, or arguments - * not matching, it is because you have instances created in the app class - * loader which don't have visibility to classes in this extension. - */ -public class ServerBuilder -{ - public Server buildServer(int port, QueryDriver driver) - { - QueryService service = new QueryServer(port, driver); - return Grpc.newServerBuilderForPort(port, InsecureServerCredentials.create()) - .addService(service) - .build(); - } -} diff --git a/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/DriverTest.java b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/DriverTest.java index 07316b47b83c..98fa27acd617 100644 --- a/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/DriverTest.java +++ b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/DriverTest.java @@ -25,7 +25,7 @@ import org.apache.druid.grpc.proto.QueryOuterClass.QueryResponse; import org.apache.druid.grpc.proto.QueryOuterClass.QueryResultFormat; import org.apache.druid.grpc.proto.QueryOuterClass.QueryStatus; -import org.apache.druid.grpc.server.QueryDriverImpl; +import org.apache.druid.grpc.server.QueryDriver; import org.apache.druid.sql.calcite.util.CalciteTests; import org.junit.BeforeClass; import org.junit.ClassRule; @@ -45,13 +45,13 @@ public class DriverTest @ClassRule public static TemporaryFolder temporaryFolder = new TemporaryFolder(); private static QueryFrameworkFixture frameworkFixture; - private static QueryDriverImpl driver; + private static QueryDriver driver; @BeforeClass public static void setup() throws IOException { frameworkFixture = new QueryFrameworkFixture(temporaryFolder.newFolder()); - driver = new QueryDriverImpl( + driver = new QueryDriver( frameworkFixture.jsonMapper(), frameworkFixture.statementFactory() ); diff --git a/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTest.java b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTest.java index 7f207675850f..eaacdee369b8 100644 --- a/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTest.java +++ b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTest.java @@ -19,16 +19,11 @@ package org.apache.druid.grpc; -import io.grpc.Grpc; -import io.grpc.InsecureChannelCredentials; -import io.grpc.ManagedChannel; -import org.apache.druid.grpc.proto.QueryGrpc; -import org.apache.druid.grpc.proto.QueryGrpc.QueryBlockingStub; import org.apache.druid.grpc.proto.QueryOuterClass.QueryRequest; import org.apache.druid.grpc.proto.QueryOuterClass.QueryResponse; import org.apache.druid.grpc.proto.QueryOuterClass.QueryResultFormat; import org.apache.druid.grpc.proto.QueryOuterClass.QueryStatus; -import org.apache.druid.grpc.server.QueryDriverImpl; +import org.apache.druid.grpc.server.QueryDriver; import org.apache.druid.grpc.server.QueryServer; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -37,52 +32,27 @@ import org.junit.rules.TemporaryFolder; import java.io.IOException; -import java.util.concurrent.TimeUnit; import static org.junit.Assert.assertEquals; +/** + * Simple test that runs the gRPC server, on top of a test SQL stack. + * Uses a simple client to send a query to the server. This is a basic + * sanity check of the gRPC stack. + */ public class GrpcQueryTest { @ClassRule - public static TemporaryFolder temporaryFolder = new TemporaryFolder(); + private static TemporaryFolder temporaryFolder = new TemporaryFolder(); private static QueryFrameworkFixture frameworkFixture; - static QueryServer server; - static TestClient client; - - public static class TestClient - { - ManagedChannel channel; - QueryBlockingStub client; - - public TestClient() - { - // Access a service running on the local machine on port 50051 - String target = "localhost:50051"; - // Create a communication channel to the server, known as a Channel. Channels are thread-safe - // and reusable. It is common to create channels at the beginning of your application and reuse - // them until the application shuts down. - // - // For the example we use plaintext insecure credentials to avoid needing TLS certificates. To - // use TLS, use TlsChannelCredentials instead. - channel = Grpc.newChannelBuilder(target, InsecureChannelCredentials.create()) - .build(); - client = QueryGrpc.newBlockingStub(channel); - } - - public void close() throws InterruptedException - { - // ManagedChannels use resources like threads and TCP connections. To prevent leaking these - // resources the channel should be shut down when it will no longer be used. If it may be used - // again leave it running. - channel.shutdownNow().awaitTermination(5, TimeUnit.SECONDS); - } - } + private static QueryServer server; + private static TestClient client; @BeforeClass public static void setup() throws IOException { frameworkFixture = new QueryFrameworkFixture(temporaryFolder.newFolder()); - QueryDriverImpl driver = new QueryDriverImpl( + QueryDriver driver = new QueryDriver( frameworkFixture.jsonMapper(), frameworkFixture.statementFactory() ); diff --git a/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTestClient.java b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTestClient.java index 70e838039445..58b178db4eeb 100644 --- a/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTestClient.java +++ b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTestClient.java @@ -19,11 +19,6 @@ package org.apache.druid.grpc; -import io.grpc.Grpc; -import io.grpc.InsecureChannelCredentials; -import io.grpc.ManagedChannel; -import org.apache.druid.grpc.proto.QueryGrpc; -import org.apache.druid.grpc.proto.QueryGrpc.QueryBlockingStub; import org.apache.druid.grpc.proto.QueryOuterClass.ColumnSchema; import org.apache.druid.grpc.proto.QueryOuterClass.QueryRequest; import org.apache.druid.grpc.proto.QueryOuterClass.QueryResponse; @@ -32,44 +27,17 @@ import org.apache.druid.java.util.common.StringUtils; import java.nio.charset.StandardCharsets; -import java.util.concurrent.TimeUnit; /** - * Super-simple client which makes a single query request and prints + * Super-simple command-line (or IDE launched) client which makes a + * single query request and prints * the response. Useful because Druid provides no other rRPC client - * to use to test the rRPC endpoint. + * to use to test the rRPC endpoint. Pass the desired query as the + * one and only command line parameter. Does not (yet) support the + * query context or query parameters. */ public class GrpcQueryTestClient { - public static class TestClient - { - ManagedChannel channel; - QueryBlockingStub client; - - public TestClient() - { - // Access a service running on the local machine on port 50051 - String target = "localhost:50051"; - // Create a communication channel to the server, known as a Channel. Channels are thread-safe - // and reusable. It is common to create channels at the beginning of your application and reuse - // them until the application shuts down. - // - // For the example we use plaintext insecure credentials to avoid needing TLS certificates. To - // use TLS, use TlsChannelCredentials instead. - channel = Grpc.newChannelBuilder(target, InsecureChannelCredentials.create()) - .build(); - client = QueryGrpc.newBlockingStub(channel); - } - - public void close() throws InterruptedException - { - // ManagedChannels use resources like threads and TCP connections. To prevent leaking these - // resources the channel should be shut down when it will no longer be used. If it may be used - // again leave it running. - channel.shutdownNow().awaitTermination(5, TimeUnit.SECONDS); - } - } - public static void main(String[] args) { if (args.length != 1) { @@ -95,7 +63,8 @@ public static void main(String[] args) System.out.println(response.getData().toString(StandardCharsets.UTF_8)); try { client.close(); - } catch (InterruptedException e) { + } + catch (InterruptedException e) { // Ignore; } } diff --git a/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/TestClient.java b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/TestClient.java new file mode 100644 index 000000000000..e32a65463ea2 --- /dev/null +++ b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/TestClient.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.grpc; + +import io.grpc.Grpc; +import io.grpc.InsecureChannelCredentials; +import io.grpc.ManagedChannel; +import org.apache.druid.grpc.proto.QueryGrpc; +import org.apache.druid.grpc.proto.QueryGrpc.QueryBlockingStub; + +import java.util.concurrent.TimeUnit; + +/** + * Super-simple test client that connects to a hardcoded (at present) + * port and allows submitting a rRPC query request and returns the response. + * The server can be in the same or another process. + */ +public class TestClient +{ + ManagedChannel channel; + QueryBlockingStub client; + + public TestClient() + { + // Access a service running on the local machine on port 50051 + String target = "localhost:50051"; + // Create a communication channel to the server, known as a Channel. Channels are thread-safe + // and reusable. It is common to create channels at the beginning of your application and reuse + // them until the application shuts down. + // + // For the example we use plaintext insecure credentials to avoid needing TLS certificates. To + // use TLS, use TlsChannelCredentials instead. + channel = Grpc.newChannelBuilder(target, InsecureChannelCredentials.create()) + .build(); + client = QueryGrpc.newBlockingStub(channel); + } + + public void close() throws InterruptedException + { + // ManagedChannels use resources like threads and TCP connections. To prevent leaking these + // resources the channel should be shut down when it will no longer be used. If it may be used + // again leave it running. + channel.shutdownNow().awaitTermination(5, TimeUnit.SECONDS); + } +} diff --git a/extensions-contrib/grpc-shaded/src/main/java/org/apache/druid/grpc/server/QueryDriver.java b/extensions-contrib/grpc-shaded/src/main/java/org/apache/druid/grpc/server/QueryDriver.java deleted file mode 100644 index 56e7b8718981..000000000000 --- a/extensions-contrib/grpc-shaded/src/main/java/org/apache/druid/grpc/server/QueryDriver.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.grpc.server; - -import org.apache.druid.grpc.proto.QueryOuterClass.QueryRequest; -import org.apache.druid.grpc.proto.QueryOuterClass.QueryResponse; - -public interface QueryDriver -{ - QueryResponse submitQuery(QueryRequest request); -} diff --git a/extensions-contrib/grpc-shaded/src/main/java/org/apache/druid/grpc/server/QueryService.java b/extensions-contrib/grpc-shaded/src/main/java/org/apache/druid/grpc/server/QueryService.java deleted file mode 100644 index 19e2cf1b18a4..000000000000 --- a/extensions-contrib/grpc-shaded/src/main/java/org/apache/druid/grpc/server/QueryService.java +++ /dev/null @@ -1,48 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.grpc.server; - -import io.grpc.stub.StreamObserver; -import org.apache.druid.grpc.proto.QueryGrpc; -import org.apache.druid.grpc.proto.QueryOuterClass.QueryRequest; -import org.apache.druid.grpc.proto.QueryOuterClass.QueryResponse; - -/** - * Implementation of the gRPC Query service. Provides a single method - * to run a query using the "driver" that holds the actual Druid SQL - * logic. - */ -class QueryService extends QueryGrpc.QueryImplBase -{ - private final QueryDriver driver; - - public QueryService(QueryDriver driver) - { - this.driver = driver; - } - - @Override - public void submitQuery(QueryRequest request, StreamObserver responseObserver) - { - QueryResponse reply = driver.submitQuery(request); - responseObserver.onNext(reply); - responseObserver.onCompleted(); - } -} diff --git a/extensions-contrib/grpc-shaded/src/main/java/org/apache/druid/grpc/server/ServerBuilder.java b/extensions-contrib/grpc-shaded/src/main/java/org/apache/druid/grpc/server/ServerBuilder.java deleted file mode 100644 index 5ab34d2cd5e0..000000000000 --- a/extensions-contrib/grpc-shaded/src/main/java/org/apache/druid/grpc/server/ServerBuilder.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.grpc.server; - -import io.grpc.BindableService; -import io.grpc.Grpc; -import io.grpc.InsecureServerCredentials; -import io.grpc.Server; - -/** - * Builds the Grpc Server given a port. This builds a Netty-based server - * internally. Done in the shaded jar so that this class can be created - * using the extension class loader, so that the gRPC classes are also - * created using that class loader which allows gRPC to find its components. - * The rest of the server is created implicitly which will use the - * the extension class loader in production, the app loader when debugging. - * If you find you are getting errors about classes not found, or arguments - * not matching, it is because you have instances created in the app class - * loader which don't have visibility to classes in this extension. - */ -public class ServerBuilder -{ - public Server buildServer(int port, BindableService service) - { - return Grpc.newServerBuilderForPort(port, InsecureServerCredentials.create()) - .addService(service) - .build(); - } -} From 5d23a4965ca1f124b07cf656cf998629a4ece402 Mon Sep 17 00:00:00 2001 From: Paul Rogers Date: Wed, 1 Feb 2023 17:01:55 -0800 Subject: [PATCH 10/43] Fix running tests from the command line --- extensions-contrib/grpc-query/pom.xml | 5 +++++ .../java/org/apache/druid/grpc/GrpcQueryTest.java | 2 +- extensions-contrib/grpc-shaded/pom.xml | 13 ++++++------- 3 files changed, 12 insertions(+), 8 deletions(-) diff --git a/extensions-contrib/grpc-query/pom.xml b/extensions-contrib/grpc-query/pom.xml index 8327d397cf5d..08cab06e50f7 100644 --- a/extensions-contrib/grpc-query/pom.xml +++ b/extensions-contrib/grpc-query/pom.xml @@ -171,6 +171,11 @@ test-jar test + + org.easymock + easymock + test + diff --git a/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTest.java b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTest.java index eaacdee369b8..a7da305d62cd 100644 --- a/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTest.java +++ b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTest.java @@ -43,7 +43,7 @@ public class GrpcQueryTest { @ClassRule - private static TemporaryFolder temporaryFolder = new TemporaryFolder(); + public static TemporaryFolder temporaryFolder = new TemporaryFolder(); private static QueryFrameworkFixture frameworkFixture; private static QueryServer server; private static TestClient client; diff --git a/extensions-contrib/grpc-shaded/pom.xml b/extensions-contrib/grpc-shaded/pom.xml index 5b5e211da84a..9b7a39652462 100644 --- a/extensions-contrib/grpc-shaded/pom.xml +++ b/extensions-contrib/grpc-shaded/pom.xml @@ -127,18 +127,17 @@ - + com.google.guava org.apache.druid.grpc.shaded.com.google.guava - com.google.common.base - org.apache.druid.grpc.shaded.com.google.common.base - - - com.google.common.util.concurrent - org.apache.druid.grpc.shaded.com.google.common.util.concurrent + com.google.common + org.apache.druid.grpc.shaded.com.google.common ${project.build.directory}/dependency-reduced-pom.xml From 131fdac25381e1d92bb7ae470ee194dc498c56da Mon Sep 17 00:00:00 2001 From: Paul Rogers Date: Thu, 2 Feb 2023 08:57:28 -0800 Subject: [PATCH 11/43] Hide gRPC shaded jar module from IDEs The gRPC shaded jar module causes issues when loaded into an IDE. This PR hides the module from Maven (and hence IDEs) by building it implicitly in the grpc-query module. It is a hack, but it works. --- .idea/misc.xml | 16 ++++++-- extensions-contrib/grpc-query/pom.xml | 31 +++++++++++++++ extensions-contrib/grpc-shaded/README.md | 48 ++++++++++++++++++++++++ pom.xml | 5 +++ 4 files changed, 96 insertions(+), 4 deletions(-) create mode 100644 extensions-contrib/grpc-shaded/README.md diff --git a/.idea/misc.xml b/.idea/misc.xml index bf2061d7392d..d76610e8ccb1 100644 --- a/.idea/misc.xml +++ b/.idea/misc.xml @@ -40,13 +40,18 @@ + @@ -84,7 +92,7 @@ - + - + \ No newline at end of file diff --git a/extensions-contrib/grpc-query/pom.xml b/extensions-contrib/grpc-query/pom.xml index 08cab06e50f7..3be6f7f5b0e1 100644 --- a/extensions-contrib/grpc-query/pom.xml +++ b/extensions-contrib/grpc-query/pom.xml @@ -180,6 +180,37 @@ + + + org.codehaus.mojo + exec-maven-plugin + + + build-shaded-jar + + exec + + initialize + + ${project.basedir}/../grpc-shaded + mvn + + install + -P + skip-static-checks,skip-tests + -Dmaven.javadoc.skip=true + + + + + org.apache.maven.plugins maven-assembly-plugin diff --git a/extensions-contrib/grpc-shaded/README.md b/extensions-contrib/grpc-shaded/README.md new file mode 100644 index 000000000000..9046a8c8ad43 --- /dev/null +++ b/extensions-contrib/grpc-shaded/README.md @@ -0,0 +1,48 @@ + + +# Shaded Jar file for the grpc-query Module + +This module produces a shaded jar that includes gRPC and the version +of Guava which gRPC needs. gRPC needs a newer Guava than Druid. +Guava is notorious for removing methods. Druid must use the version +of Guava which Hadoop 2 needs. That version omits methods which gRPC +needs. But, if we use the gRPC Guava version, then that version omits +methods which Hadoop 2 needs. The solution is to shade rRPC and its +required Guava version. + +Once we do that, we realize that we cannot use the shaeded jar to +generate Protobuf code: the generated code refers to Guava in its +unshaded location, causing more version conflicts. To resolve this +we first generate the Java Protobuf code, then include that in the +shaded jar, which relocates the offending Guava references. + +The result works as we want, except that it causes grief in an IDE. +Basically, the IDE doesn't know about the shaded jar: it pulls in +the dependencies of this module directly. The result is the very +Guava conflicts we want to avoid. To fix this, we **do not** +include this module in the root-level `pom.xml` file. Instead the +`grpc-query` module uses a Maven exec plugin to do the build of +this module. The result in a full build is the same, with the +benefit that this module is invisible to IDEs. + +The one drawback of this odd structure is that this project +will not see requests to run static checks. That should be OK: +the code here is quite small and won't often change. Still, if there +is a better solution, we can switch to it instead. diff --git a/pom.xml b/pom.xml index 45189974bbf2..0b842b418e37 100644 --- a/pom.xml +++ b/pom.xml @@ -215,7 +215,12 @@ extensions-contrib/prometheus-emitter extensions-contrib/opentelemetry-emitter extensions-contrib/kubernetes-overlord-extensions + extensions-contrib/grpc-query distribution From 3df151aac11c172d4a3c8a245e5a61e0906c7518 Mon Sep 17 00:00:00 2001 From: Courage Noko Date: Mon, 6 Feb 2023 16:45:32 -0500 Subject: [PATCH 12/43] grpc protobufwriter --- extensions-contrib/grpc-query/pom.xml | 27 ++++ .../grpc/client/GrpcResponseHandler.java | 96 ++++++++++++++ .../grpc/server/ProtobufTransformer.java | 105 +++++++++++++++ .../druid/grpc/server/ProtobufWriter.java | 120 ++++++++++++++++++ .../apache/druid/grpc/server/QueryDriver.java | 30 ++++- .../apache/druid/grpc/server/QueryServer.java | 6 +- .../org/apache/druid/grpc/GrpcQueryTest.java | 18 +++ .../grpc-query/src/test/proto/results.proto | 32 +++++ 8 files changed, 424 insertions(+), 10 deletions(-) create mode 100644 extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/client/GrpcResponseHandler.java create mode 100644 extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/ProtobufTransformer.java create mode 100644 extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/ProtobufWriter.java create mode 100644 extensions-contrib/grpc-query/src/test/proto/results.proto diff --git a/extensions-contrib/grpc-query/pom.xml b/extensions-contrib/grpc-query/pom.xml index 3be6f7f5b0e1..7547c34840b5 100644 --- a/extensions-contrib/grpc-query/pom.xml +++ b/extensions-contrib/grpc-query/pom.xml @@ -179,6 +179,13 @@ + + + kr.motd.maven + os-maven-plugin + 1.5.0.Final + + + + + 4.0.0 + org.apache.druid.extensions.contrib + grpc-query-it + grpc-query-it + grpc-query integration tests + + + org.apache.druid + druid + 26.0.0-SNAPSHOT + ../../pom.xml + + + + MustSetACategory + ../../integration-tests-ex/cases + + + + + org.apache.druid.integration-tests + druid-it-cases + ${project.parent.version} + test-jar + test + + + org.apache.druid.extensions.contrib + druid-shaded-grpc + ${project.parent.version} + + + junit + junit + test + + + pl.pragmatists + JUnitParams + test + + + + + + IT-GrpcQuery + + false + + + GrpcQuery + + + + docker-tests + + false + + + + + maven-failsafe-plugin + 3.0.0-M7 + + + + org.apache.maven.surefire + surefire-junit47 + 3.0.0-M7 + + + + + functional-test + integration-test + + integration-test + + + + False + + org.apache.druid.testsEx.categories.${it.category} + + + + verify + verify + + verify + + + + + + + org.codehaus.mojo + exec-maven-plugin + + + start-cluster + pre-integration-test + + exec + + + ${project.basedir} + bash + + ${it.base.dir}/cluster.sh + up + ${it.category} + + + + + + stop-cluster + post-integration-test + + exec + + + ${project.basedir} + bash + + ${it.base.dir}/cluster.sh + down + ${it.category} + + + + + + + + + + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + **/IT* + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + + org.glassfish.hk2.external:jakarta.inject + + + + mysql:mysql-connector-java:jar + + + + + org.owasp + dependency-check-maven + + true + + + + + + diff --git a/extensions-contrib/grpc-query-it/src/test/java/org/apache/druid/testsEx/categories/GrpcQuery.java b/extensions-contrib/grpc-query-it/src/test/java/org/apache/druid/testsEx/categories/GrpcQuery.java new file mode 100644 index 000000000000..45033938b8db --- /dev/null +++ b/extensions-contrib/grpc-query-it/src/test/java/org/apache/druid/testsEx/categories/GrpcQuery.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.categories; + +public class GrpcQuery +{ +} diff --git a/extensions-contrib/grpc-query-it/src/test/java/org/apache/druid/testsEx/grpc/GrpcQueryTest.java b/extensions-contrib/grpc-query-it/src/test/java/org/apache/druid/testsEx/grpc/GrpcQueryTest.java new file mode 100644 index 000000000000..f98b5fa4967b --- /dev/null +++ b/extensions-contrib/grpc-query-it/src/test/java/org/apache/druid/testsEx/grpc/GrpcQueryTest.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.grpc; + +import org.apache.druid.testsEx.categories.GrpcQuery; +import org.apache.druid.testsEx.config.DruidTestRunner; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +@RunWith(DruidTestRunner.class) +@Category(GrpcQuery.class) +public class GrpcQueryTest +{ +} diff --git a/extensions-contrib/grpc-query-it/templates/GrpcQuery.py b/extensions-contrib/grpc-query-it/templates/GrpcQuery.py new file mode 100644 index 000000000000..ec6778a33413 --- /dev/null +++ b/extensions-contrib/grpc-query-it/templates/GrpcQuery.py @@ -0,0 +1,22 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import sys, os.path +lib_dir = os.path.abspath("../../../integration-tests-ex/cases/templates") +print(lib_dir) +sys.path.append(lib_dir) +from template import BaseTemplate, generate + +generate(__file__, BaseTemplate()) diff --git a/extensions-contrib/grpc-query/pom.xml b/extensions-contrib/grpc-query/pom.xml index 7547c34840b5..ba0674065637 100644 --- a/extensions-contrib/grpc-query/pom.xml +++ b/extensions-contrib/grpc-query/pom.xml @@ -32,12 +32,12 @@ ../../pom.xml - - 1.52.0 - 3.2.1 - - + + org.apache.druid.extensions.contrib + druid-shaded-grpc + ${project.parent.version} + org.apache.druid druid-core @@ -202,18 +202,18 @@ build-shaded-jar - exec + exec initialize - ${project.basedir}/../grpc-shaded - mvn - - install - -P - skip-static-checks,skip-tests - -Dmaven.javadoc.skip=true - + ${project.basedir}/../grpc-shaded + mvn + + install + -P + skip-static-checks,skip-tests + -Dmaven.javadoc.skip=true + @@ -243,24 +243,24 @@ - org.xolstice.maven.plugins - protobuf-maven-plugin - 0.6.1 - - com.google.protobuf:protoc:3.21.7:exe:${os.detected.classifier} - grpc-java - io.grpc:protoc-gen-grpc-java:1.52.0:exe:${os.detected.classifier} - - - - - compile - compile-custom - test-compile - test-compile-custom - - - + org.xolstice.maven.plugins + protobuf-maven-plugin + 0.6.1 + + com.google.protobuf:protoc:3.21.7:exe:${os.detected.classifier} + grpc-java + io.grpc:protoc-gen-grpc-java:1.52.0:exe:${os.detected.classifier} + + + + + compile + compile-custom + test-compile + test-compile-custom + + + diff --git a/integration-tests-ex/cases/cluster.sh b/integration-tests-ex/cases/cluster.sh index f7c6cb92f0bf..87c58e72d66e 100755 --- a/integration-tests-ex/cases/cluster.sh +++ b/integration-tests-ex/cases/cluster.sh @@ -27,15 +27,16 @@ set -e # Enable for debugging #set -x -export MODULE_DIR=$(cd $(dirname $0) && pwd) +export BASE_MODULE_DIR=$(cd $(dirname $0) && pwd) + +# The location of the tests, if different than the +export MODULE_DIR=${IT_MODULE_DIR:-$BASE_MODULE_DIR} function usage { cat <&2 exit 1 @@ -133,7 +134,7 @@ function build_shared_dir { mkdir -p $SHARED_DIR/db mkdir -p $SHARED_DIR/kafka mkdir -p $SHARED_DIR/resources - cp $MODULE_DIR/assets/log4j2.xml $SHARED_DIR/resources + cp $BASE_MODULE_DIR/assets/log4j2.xml $SHARED_DIR/resources # Permissions in some build setups are screwed up. See above. The user # which runs Docker does not have permission to write into the /shared # directory. Force ownership to allow writing. @@ -153,7 +154,7 @@ function docker_file { pushd $TEMPLATE_DIR > /dev/null python3 $TEMPLATE_SCRIPT popd > /dev/null - cp -r $MODULE_DIR/cluster/Common $TARGET_DIR/cluster + cp -r $BASE_MODULE_DIR/cluster/Common $TARGET_DIR/cluster else # Else, use the existing non-template file in place. if [ ! -d $CLUSTER_DIR ]; then @@ -219,17 +220,6 @@ set -e # Print environment for debugging #env -# Determine if docker-compose is available. If not, assume Docker supports -# the compose subcommand -set +e -if which docker-compose > /dev/null -then - DOCKER_COMPOSE='docker-compose' -else - DOCKER_COMPOSE='docker compose' -fi -set -e - case $CMD in "-h" ) usage @@ -238,17 +228,11 @@ case $CMD in usage $DOCKER_COMPOSE help ;; - "prepare" ) - check_env_file - category $* - build_shared_dir - docker_file - ;; "gen" ) category $* build_shared_dir docker_file - echo "Generated file is in $COMPOSE_DIR" + echo "Generated file is $COMPOSE_DIR/docker-compose.yaml" ;; "up" ) check_env_file diff --git a/integration-tests-ex/cases/pom.xml b/integration-tests-ex/cases/pom.xml index 84ec766b99db..b13bb6fc66ac 100644 --- a/integration-tests-ex/cases/pom.xml +++ b/integration-tests-ex/cases/pom.xml @@ -21,517 +21,512 @@ - 4.0.0 + 4.0.0 - org.apache.druid.integration-tests - druid-it-cases - druid-it-cases - New Integration Tests + org.apache.druid.integration-tests + druid-it-cases + druid-it-cases + New Integration Tests - - org.apache.druid - druid - 26.0.0-SNAPSHOT - ../../pom.xml - + + org.apache.druid + druid + 26.0.0-SNAPSHOT + ../../pom.xml + - - MustSetACategory - + + MustSetACategory + - - - org.apache.druid - druid-integration-tests - ${project.parent.version} - - - - org.apache.druid - druid-integration-tests - ${project.parent.version} - tests - test-jar - test - - - org.apache.druid - druid-processing - ${project.parent.version} - - - org.apache.druid - druid-server - ${project.parent.version} - - - org.apache.druid - druid-services - ${project.parent.version} - - - org.apache.druid - druid-indexing-service - ${project.parent.version} - - - com.google.inject - guice - - - com.google.inject.extensions - guice-multibindings - - - org.apache.curator - curator-framework - - - com.google.guava - guava - - - com.fasterxml.jackson.dataformat - jackson-dataformat-yaml - - - com.fasterxml.jackson.core - jackson-annotations - - - com.fasterxml.jackson.core - jackson-databind - - - com.fasterxml.jackson.core - jackson-core - - - joda-time - joda-time - - - commons-io - commons-io - - - com.github.docker-java - docker-java-core - - - com.github.docker-java - docker-java-api - - - com.github.docker-java - docker-java-transport-netty - - - com.github.docker-java - docker-java - - - com.github.docker-java - docker-java-transport-jersey - - - io.netty - netty-transport-native-kqueue - - - - - com.amazonaws - aws-java-sdk-core - ${aws.sdk.version} - - - com.google.api-client - google-api-client - ${com.google.apis.client.version} - provided - - - com.google.apis - google-api-services-storage - ${com.google.apis.storage.version} - - - com.google.api-client - google-api-client - - - - - com.microsoft.azure - azure-storage - 8.6.0 - - - org.slf4j - slf4j-api - - - com.fasterxml.jackson.core - jackson-core - - - org.apache.commons - commons-lang3 - - - com.google.guava - guava - - - - - com.google.http-client - google-http-client - ${com.google.apis.client.version} - provided - - - com.google.http-client - google-http-client-jackson2 - ${com.google.apis.client.version} - provided - - - - mysql - mysql-connector-java - ${mysql.version} - runtime - - - org.mariadb.jdbc - mariadb-java-client - ${mariadb.version} - runtime - - - org.jdbi - jdbi - - - org.apache.druid.extensions - mysql-metadata-storage - ${project.parent.version} - - - org.apache.druid.extensions - druid-azure-extensions - ${project.parent.version} - provided - - - com.amazonaws - aws-java-sdk-s3 - ${aws.sdk.version} - provided - - - org.apache.druid.extensions - druid-s3-extensions - ${project.parent.version} - provided - - - org.apache.druid.extensions - druid-hdfs-storage - ${project.parent.version} - provided - - - com.amazonaws - aws-java-sdk-bundle - - - - - org.apache.druid.extensions - druid-google-extensions - ${project.parent.version} - provided - - - org.apache.druid - druid-gcp-common - ${project.parent.version} - provided - + + + org.apache.druid + druid-integration-tests + ${project.parent.version} + + + + org.apache.druid + druid-integration-tests + ${project.parent.version} + tests + test-jar + test + + + org.apache.druid + druid-processing + ${project.parent.version} + + + org.apache.druid + druid-server + ${project.parent.version} + + + org.apache.druid + druid-services + ${project.parent.version} + + + org.apache.druid + druid-indexing-service + ${project.parent.version} + + + com.google.inject + guice + + + com.google.inject.extensions + guice-multibindings + + + org.apache.curator + curator-framework + + + com.google.guava + guava + + + com.fasterxml.jackson.dataformat + jackson-dataformat-yaml + + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-databind + + + com.fasterxml.jackson.core + jackson-core + + + joda-time + joda-time + + + commons-io + commons-io + + + com.github.docker-java + docker-java-core + + + com.github.docker-java + docker-java-api + + + com.github.docker-java + docker-java-transport-netty + + + com.github.docker-java + docker-java + + + com.github.docker-java + docker-java-transport-jersey + + + io.netty + netty-transport-native-kqueue + + + + + com.amazonaws + aws-java-sdk-core + + + com.google.api-client + google-api-client + + + com.google.apis + google-api-services-storage + ${com.google.apis.storage.version} + + + com.google.api-client + google-api-client + + + + + com.microsoft.azure + azure-storage + 8.6.0 + + + org.slf4j + slf4j-api + + + com.fasterxml.jackson.core + jackson-core + + + org.apache.commons + commons-lang3 + + + com.google.guava + guava + + + + + com.google.http-client + google-http-client + + + com.google.http-client + google-http-client-jackson2 + + + + mysql + mysql-connector-java + ${mysql.version} + runtime + + + org.mariadb.jdbc + mariadb-java-client + ${mariadb.version} + runtime + + + org.jdbi + jdbi + + + org.apache.druid.extensions + mysql-metadata-storage + ${project.parent.version} + + + org.apache.druid.extensions + druid-azure-extensions + ${project.parent.version} + + + com.amazonaws + aws-java-sdk-s3 + + + org.apache.druid.extensions + druid-s3-extensions + ${project.parent.version} + + + org.apache.druid.extensions + druid-hdfs-storage + ${project.parent.version} + provided + + + com.amazonaws + aws-java-sdk-bundle + + + + + org.apache.druid.extensions + druid-google-extensions + ${project.parent.version} + + + org.apache.druid + druid-gcp-common + ${project.parent.version} + - - org.apache.druid - druid-sql - ${project.parent.version} - - - org.apache.druid.extensions - druid-multi-stage-query - ${project.parent.version} - provided - - - org.apache.druid.extensions - druid-catalog - ${project.parent.version} - provided - - - org.apache.commons - commons-lang3 - - - io.netty - netty - - - org.apache.kafka - kafka-clients - ${apache.kafka.version} - test - - - com.google.code.findbugs - jsr305 - - - junit - junit - test - - - pl.pragmatists - JUnitParams - test - - - javax.ws.rs - jsr311-api - - - org.apache.curator - curator-client - 5.4.0 - - + + org.apache.druid + druid-sql + ${project.parent.version} + + + org.apache.druid.extensions + druid-multi-stage-query + ${project.parent.version} + + + org.apache.druid.extensions + druid-catalog + ${project.parent.version} + + + org.apache.commons + commons-lang3 + + + io.netty + netty + + + org.apache.kafka + kafka-clients + ${apache.kafka.version} + test + + + com.google.code.findbugs + jsr305 + + + junit + junit + test + + + pl.pragmatists + JUnitParams + test + + + javax.ws.rs + jsr311-api + + + org.apache.curator + curator-client + + - - + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + **/IT* + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + + org.glassfish.hk2.external:jakarta.inject + + + + mysql:mysql-connector-java:jar + + + + + org.owasp + dependency-check-maven + + true + + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + + + + + + + + + IT-HighAvailability + + false + + + HighAvailability + + + + IT-BatchIndex + + false + + + BatchIndex + + + + IT-InputSource + + false + + + InputSource + + + + IT-InputFormat + + false + + + InputFormat + + + + IT-AzureDeepStorage + + false + + + AzureDeepStorage + + + + IT-MultiStageQuery + + false + + + MultiStageQuery + + + + IT-MultiStageQueryWithMM + + false + + + MultiStageQueryWithMM + + + + IT-Catalog + + false + + + Catalog + + + + IT-S3DeepStorage + + false + + + S3DeepStorage + + + + IT-GcsDeepStorage + + false + + + GcsDeepStorage + + + + docker-tests + + false + + - - org.apache.maven.plugins - maven-surefire-plugin + + maven-failsafe-plugin + 3.0.0-M7 + + + + org.apache.maven.surefire + surefire-junit47 + 3.0.0-M7 + + + + + functional-test + integration-test + + integration-test + - - **/IT* - - - - - org.apache.maven.plugins - maven-dependency-plugin + + False + + org.apache.druid.testsEx.categories.${it.category} + + + + verify + verify + + verify + + + + + + + org.codehaus.mojo + exec-maven-plugin + + + start-cluster + pre-integration-test + + exec + - - - org.glassfish.hk2.external:jakarta.inject - - - - mysql:mysql-connector-java:jar - + ${project.basedir} + bash + + cluster.sh + up + ${it.category} + - - - org.owasp - dependency-check-maven + + + + stop-cluster + post-integration-test + + exec + - true + ${project.basedir} + bash + + cluster.sh + down + ${it.category} + - + + + - - - - - IT-HighAvailability - - false - - - HighAvailability - - - - IT-BatchIndex - - false - - - BatchIndex - - - - IT-InputSource - - false - - - InputSource - - - - IT-InputFormat - - false - - - InputFormat - - - - IT-AzureDeepStorage - - false - - - AzureDeepStorage - - - - IT-MultiStageQuery - - false - - - MultiStageQuery - - - - IT-MultiStageQueryWithMM - - false - - - MultiStageQueryWithMM - - - - IT-Catalog - - false - - - Catalog - - - - IT-S3DeepStorage - - false - - - S3DeepStorage - - - - IT-GcsDeepStorage - - false - - - GcsDeepStorage - - - - docker-tests - - false - - - - - maven-failsafe-plugin - 3.0.0-M7 - - - - org.apache.maven.surefire - surefire-junit47 - 3.0.0-M7 - - - - - functional-test - integration-test - - integration-test - - - - False - - org.apache.druid.testsEx.categories.${it.category} - - - - verify - verify - - verify - - - - - - - org.codehaus.mojo - exec-maven-plugin - - - start-cluster - pre-integration-test - - exec - - - ${project.basedir} - bash - - cluster.sh - up - ${it.category} - - - - - - stop-cluster - post-integration-test - - exec - - - ${project.basedir} - bash - - cluster.sh - down - ${it.category} - - - - - - - - - + + + diff --git a/integration-tests-ex/cases/templates/template.py b/integration-tests-ex/cases/templates/template.py index 1be24ab032e4..882d5532d8fa 100644 --- a/integration-tests-ex/cases/templates/template.py +++ b/integration-tests-ex/cases/templates/template.py @@ -23,7 +23,7 @@ PyYaml does the grunt work of converting the data structure to the YAML file. ''' -import yaml, os, os.path +import yaml, os from pathlib import Path # Constants used frequently in the template. @@ -49,16 +49,21 @@ def generate(template_path, template): ''' # Compute the cluster (test category) name from the template path which - # we assume to be module//