From 4729279affc0f077bc8cd4ca59ca18502d55ae34 Mon Sep 17 00:00:00 2001 From: cryptoe Date: Tue, 13 Jun 2023 18:33:46 +0530 Subject: [PATCH 01/15] Adding Interactive API's for MSQ engine --- .../apache/druid/msq/guice/SqlTaskModule.java | 9 +- .../druid/msq/indexing/MSQControllerTask.java | 3 +- .../druid/msq/sql/MSQTaskQueryMaker.java | 5 + .../druid/msq/sql/MSQTaskSqlEngine.java | 5 +- .../druid/msq/sql/SqlStatementState.java | 28 + .../apache/druid/msq/sql/SqlTaskStatus.java | 1 + .../druid/msq/sql/entity/ColNameAndType.java | 83 ++ .../msq/sql/entity/ResultSetInformation.java | 141 ++++ .../msq/sql/entity/SqlStatementResult.java | 155 ++++ .../sql/resources/SqlStatementResource.java | 753 ++++++++++++++++++ .../sql/{ => resources}/SqlTaskResource.java | 4 +- .../indexing/report/MSQTaskReportTest.java | 2 +- .../msq/sql/SqlStatementResourceTest.java | 490 ++++++++++++ .../org/apache/druid/query/ExecutionMode.java | 28 + .../org/apache/druid/query/QueryContexts.java | 17 +- .../druid/sql/http/SqlResourceTest.java | 2 +- 16 files changed, 1719 insertions(+), 7 deletions(-) create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlStatementState.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ColNameAndType.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ResultSetInformation.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/SqlStatementResult.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java rename extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/{ => resources}/SqlTaskResource.java (98%) create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java create mode 100644 processing/src/main/java/org/apache/druid/query/ExecutionMode.java diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/SqlTaskModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/SqlTaskModule.java index d8f8b06f9b9f..bcd8abfbb87c 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/SqlTaskModule.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/SqlTaskModule.java @@ -21,12 +21,16 @@ import com.fasterxml.jackson.databind.Module; import com.google.inject.Binder; +import org.apache.druid.client.indexing.HttpIndexingServiceClient; +import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.discovery.NodeRole; import org.apache.druid.guice.Jerseys; +import org.apache.druid.guice.LazySingleton; import org.apache.druid.guice.LifecycleModule; import org.apache.druid.guice.annotations.LoadScope; import org.apache.druid.initialization.DruidModule; -import org.apache.druid.msq.sql.SqlTaskResource; +import org.apache.druid.msq.sql.resources.SqlStatementResource; +import org.apache.druid.msq.sql.resources.SqlTaskResource; import java.util.Collections; import java.util.List; @@ -43,6 +47,9 @@ public void configure(Binder binder) // Force eager initialization. LifecycleModule.register(binder, SqlTaskResource.class); Jerseys.addResource(binder, SqlTaskResource.class); + LifecycleModule.register(binder, SqlStatementResource.class); + binder.bind(IndexingServiceClient.class).to(HttpIndexingServiceClient.class).in(LazySingleton.class); + Jerseys.addResource(binder, SqlStatementResource.class); } @Override diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java index 3265f2640af6..fb1bc13384cc 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java @@ -30,6 +30,7 @@ import com.google.inject.Injector; import com.google.inject.Key; import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.druid.client.indexing.ClientTaskQuery; import org.apache.druid.guice.annotations.EscalatedGlobal; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.TaskLock; @@ -60,7 +61,7 @@ import java.util.Set; @JsonTypeName(MSQControllerTask.TYPE) -public class MSQControllerTask extends AbstractTask +public class MSQControllerTask extends AbstractTask implements ClientTaskQuery { public static final String TYPE = "query_controller"; public static final String DUMMY_DATASOURCE_FOR_SELECT = "__query_select"; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java index cde4906b193d..0bfb4341b92e 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java @@ -78,6 +78,8 @@ public class MSQTaskQueryMaker implements QueryMaker private static final String DESTINATION_DATASOURCE = "dataSource"; private static final String DESTINATION_REPORT = "taskReport"; + public static final String USER_KEY = "__user"; + private static final Granularity DEFAULT_SEGMENT_GRANULARITY = Granularities.ALL; private final String targetDataSource; @@ -116,6 +118,9 @@ public QueryResponse runQuery(final DruidQuery druidQuery) // Native query context: sqlQueryContext plus things that we add prior to creating a controller task. final Map nativeQueryContext = new HashMap<>(sqlQueryContext.asMap()); + // adding user + nativeQueryContext.put(USER_KEY, plannerContext.getAuthenticationResult().getIdentity()); + final String msqMode = MultiStageQueryContext.getMSQMode(sqlQueryContext); if (msqMode != null) { MSQMode.populateDefaultQueryContext(msqMode, nativeQueryContext); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskSqlEngine.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskSqlEngine.java index 94c2532ca793..0788e69929bf 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskSqlEngine.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskSqlEngine.java @@ -296,7 +296,10 @@ private static void validateLimitAndOffset(final RelNode topRel, final boolean l private static RelDataType getMSQStructType(RelDataTypeFactory typeFactory) { return typeFactory.createStructType( - ImmutableList.of(Calcites.createSqlType(typeFactory, SqlTypeName.VARCHAR)), + ImmutableList.of( + Calcites.createSqlType(typeFactory, SqlTypeName.VARCHAR), + Calcites.createSqlType(typeFactory, SqlTypeName.VARCHAR) + ), TASK_STRUCT_FIELD_NAMES ); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlStatementState.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlStatementState.java new file mode 100644 index 000000000000..6b45272394d2 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlStatementState.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.msq.sql; + +public enum SqlStatementState +{ + ACCEPTED, + RUNNING, + SUCCESS, + FAILED +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlTaskStatus.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlTaskStatus.java index d47533315e27..386d2b388122 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlTaskStatus.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlTaskStatus.java @@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import org.apache.druid.indexer.TaskState; +import org.apache.druid.msq.sql.resources.SqlTaskResource; import org.apache.druid.query.QueryException; import javax.annotation.Nullable; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ColNameAndType.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ColNameAndType.java new file mode 100644 index 000000000000..29c1a686a2c0 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ColNameAndType.java @@ -0,0 +1,83 @@ +/* + * 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.msq.sql.entity; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Objects; + + +public class ColNameAndType +{ + + private final String colName; + private final String sqlTypeName; + + @JsonCreator + public ColNameAndType(@JsonProperty("name") String colName, @JsonProperty("type") String sqlTypeName) + { + + this.colName = colName; + this.sqlTypeName = sqlTypeName; + } + + @JsonProperty("name") + public String getColName() + { + return colName; + } + + @JsonProperty("type") + public String getSqlTypeName() + { + return sqlTypeName; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ColNameAndType that = (ColNameAndType) o; + return Objects.equals(colName, that.colName) && Objects.equals(sqlTypeName, that.sqlTypeName); + } + + @Override + public int hashCode() + { + return Objects.hash(colName, sqlTypeName); + } + + @Override + public String toString() + { + return "ColNameAndType{" + + "colName='" + colName + '\'' + + ", sqlTypeName='" + sqlTypeName + '\'' + + '}'; + } +} + + diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ResultSetInformation.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ResultSetInformation.java new file mode 100644 index 000000000000..d4533e1c9daa --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ResultSetInformation.java @@ -0,0 +1,141 @@ +/* + * 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.msq.sql.entity; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.sql.http.ResultFormat; + +import javax.annotation.Nullable; +import java.util.List; +import java.util.Objects; + +public class ResultSetInformation +{ + + @Nullable + private final Long totalRows; + @Nullable + private final Long totalSize; + + private final ResultFormat resultFormat; + + private final Boolean header; + + @Nullable + private final List records; + + @JsonCreator + public ResultSetInformation( + @Nullable + @JsonProperty ResultFormat resultFormat, + @Nullable + @JsonProperty Boolean header, + @JsonProperty @Nullable Long totalRows, + @JsonProperty @Nullable Long totalSize, + @JsonProperty("sampleRecords") @Nullable + List records + ) + { + this.totalRows = totalRows; + this.totalSize = totalSize; + this.resultFormat = resultFormat; + this.header = header; + this.records = records; + } + + @Nullable + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public Long getTotalRows() + { + return totalRows; + } + + @Nullable + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public Long getTotalSize() + { + return totalSize; + } + + @JsonProperty + @Nullable + @JsonInclude(JsonInclude.Include.NON_NULL) + public ResultFormat getResultFormat() + { + return resultFormat; + } + + @JsonProperty + @Nullable + @JsonInclude(JsonInclude.Include.NON_NULL) + public Boolean getHeader() + { + return header; + } + + @Nullable + @JsonProperty("sampleRecords") + @JsonInclude(JsonInclude.Include.NON_NULL) + public List getRecords() + { + return records; + } + + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ResultSetInformation that = (ResultSetInformation) o; + return Objects.equals(totalRows, that.totalRows) + && Objects.equals(totalSize, that.totalSize) + && resultFormat == that.resultFormat + && Objects.equals(header, that.header) + && Objects.equals(records, that.records); + } + + @Override + public int hashCode() + { + return Objects.hash(totalRows, totalSize, resultFormat, header, records); + } + + @Override + public String toString() + { + return "ResultSetInformation{" + + "totalRows=" + totalRows + + ", totalSize=" + totalSize + + ", resultFormat=" + resultFormat + + ", header=" + header + + ", records=" + records + + '}'; + } +} + diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/SqlStatementResult.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/SqlStatementResult.java new file mode 100644 index 000000000000..afede8cac2a3 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/SqlStatementResult.java @@ -0,0 +1,155 @@ +/* + * 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.msq.sql.entity; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.msq.sql.SqlStatementState; +import org.joda.time.DateTime; + +import javax.annotation.Nullable; +import java.util.List; +import java.util.Objects; + +public class SqlStatementResult +{ + + private final String queryId; + + private final SqlStatementState state; + + private final DateTime createdAt; + + @Nullable + private final List sqlRowSignature; + + @Nullable + private final Long durationInMs; + + @Nullable + private final ResultSetInformation resultSetInformation; + + + @JsonCreator + public SqlStatementResult( + @JsonProperty("queryId") + String queryId, + @JsonProperty("state") + SqlStatementState state, + @JsonProperty("createdAt") + DateTime createdAt, + @Nullable @JsonProperty("schema") + List sqlRowSignature, + @Nullable @JsonProperty("durationInMs") + Long durationInMs, + @Nullable @JsonProperty("result") + ResultSetInformation resultSetInformation + + ) + { + this.queryId = queryId; + this.state = state; + this.createdAt = createdAt; + this.sqlRowSignature = sqlRowSignature; + this.durationInMs = durationInMs; + this.resultSetInformation = resultSetInformation; + } + + @JsonProperty + public String getQueryId() + { + return queryId; + } + + @JsonProperty + public SqlStatementState getState() + { + return state; + } + + @JsonProperty + public DateTime getCreatedAt() + { + return createdAt; + } + + @JsonProperty("schema") + @Nullable + @JsonInclude(JsonInclude.Include.NON_NULL) + public List getSqlRowSignature() + { + return sqlRowSignature; + } + + @JsonProperty + @Nullable + @JsonInclude(JsonInclude.Include.NON_NULL) + public Long getDurationInMs() + { + return durationInMs; + } + + @JsonProperty + @Nullable + @JsonInclude(JsonInclude.Include.NON_NULL) + public ResultSetInformation getResultSetInformation() + { + return resultSetInformation; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + SqlStatementResult that = (SqlStatementResult) o; + return Objects.equals(queryId, that.queryId) && state == that.state && Objects.equals( + createdAt, + that.createdAt + ) && Objects.equals(sqlRowSignature, that.sqlRowSignature) && Objects.equals( + durationInMs, + that.durationInMs + ) && Objects.equals(resultSetInformation, that.resultSetInformation); + } + + @Override + public int hashCode() + { + return Objects.hash(queryId, state, createdAt, sqlRowSignature, durationInMs, resultSetInformation); + } + + @Override + public String toString() + { + return "StatementSqlResult{" + + "queryId='" + queryId + '\'' + + ", state=" + state + + ", createdAt=" + createdAt + + ", sqlRowSignature=" + sqlRowSignature + + ", durationInMs=" + durationInMs + + ", resultSetInformation=" + resultSetInformation + + '}'; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java new file mode 100644 index 000000000000..a33c43a47d30 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java @@ -0,0 +1,753 @@ +/* + * 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.msq.sql.resources; + + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import com.google.common.io.CountingOutputStream; +import com.google.inject.Inject; +import org.apache.calcite.plan.RelOptPlanner; +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.druid.client.indexing.IndexingServiceClient; +import org.apache.druid.client.indexing.TaskPayloadResponse; +import org.apache.druid.client.indexing.TaskStatusResponse; +import org.apache.druid.common.exception.SanitizableException; +import org.apache.druid.guice.annotations.MSQ; +import org.apache.druid.indexer.TaskLocation; +import org.apache.druid.indexer.TaskState; +import org.apache.druid.indexer.TaskStatusPlus; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.RE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Yielder; +import org.apache.druid.java.util.common.guava.Yielders; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.msq.indexing.MSQControllerTask; +import org.apache.druid.msq.indexing.TaskReportMSQDestination; +import org.apache.druid.msq.sql.MSQTaskQueryMaker; +import org.apache.druid.msq.sql.MSQTaskSqlEngine; +import org.apache.druid.msq.sql.SqlStatementState; +import org.apache.druid.msq.sql.entity.ColNameAndType; +import org.apache.druid.msq.sql.entity.ResultSetInformation; +import org.apache.druid.msq.sql.entity.SqlStatementResult; +import org.apache.druid.query.BadQueryException; +import org.apache.druid.query.ExecutionMode; +import org.apache.druid.query.QueryCapacityExceededException; +import org.apache.druid.query.QueryContexts; +import org.apache.druid.query.QueryException; +import org.apache.druid.query.QueryInterruptedException; +import org.apache.druid.query.QueryTimeoutException; +import org.apache.druid.query.QueryUnsupportedException; +import org.apache.druid.query.ResourceLimitExceededException; +import org.apache.druid.server.QueryResponse; +import org.apache.druid.server.initialization.ServerConfig; +import org.apache.druid.server.security.Access; +import org.apache.druid.server.security.AuthenticationResult; +import org.apache.druid.server.security.AuthorizationUtils; +import org.apache.druid.server.security.AuthorizerMapper; +import org.apache.druid.server.security.ForbiddenException; +import org.apache.druid.sql.DirectStatement; +import org.apache.druid.sql.HttpStatement; +import org.apache.druid.sql.SqlPlanningException; +import org.apache.druid.sql.SqlRowTransformer; +import org.apache.druid.sql.SqlStatementFactory; +import org.apache.druid.sql.calcite.planner.ColumnMappings; +import org.apache.druid.sql.http.ResultFormat; +import org.apache.druid.sql.http.SqlQuery; +import org.apache.druid.sql.http.SqlResource; +import org.apache.http.HttpStatus; + +import javax.servlet.http.HttpServletRequest; +import javax.ws.rs.Consumes; +import javax.ws.rs.DELETE; +import javax.ws.rs.GET; +import javax.ws.rs.POST; +import javax.ws.rs.Path; +import javax.ws.rs.PathParam; +import javax.ws.rs.Produces; +import javax.ws.rs.QueryParam; +import javax.ws.rs.core.Context; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; +import javax.ws.rs.core.StreamingOutput; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Optional; + + +@Path("/druid/v2/sql/statements/") +public class SqlStatementResource +{ + + private static final Logger log = new Logger(SqlStatementResource.class); + private final SqlStatementFactory msqSqlStatementFactory; + private final ServerConfig serverConfig; + private final AuthorizerMapper authorizerMapper; + private final ObjectMapper jsonMapper; + private final IndexingServiceClient overlordClient; + + + @Inject + public SqlStatementResource( + final @MSQ SqlStatementFactory msqSqlStatementFactory, + final ServerConfig serverConfig, + final AuthorizerMapper authorizerMapper, + final ObjectMapper jsonMapper, + final IndexingServiceClient overlordClient + ) + { + this.msqSqlStatementFactory = msqSqlStatementFactory; + this.serverConfig = serverConfig; + this.authorizerMapper = authorizerMapper; + this.jsonMapper = jsonMapper; + this.overlordClient = overlordClient; + } + + + @GET + @Path("/enabled") + @Produces(MediaType.APPLICATION_JSON) + public Response doGetEnabled(@Context final HttpServletRequest request) + { + // All authenticated users are authorized for this API: check an empty resource list. + final Access authResult = AuthorizationUtils.authorizeAllResourceActions( + request, + Collections.emptyList(), + authorizerMapper + ); + + if (!authResult.isAllowed()) { + throw new ForbiddenException(authResult.toString()); + } + + return Response.ok(ImmutableMap.of("enabled", true)).build(); + } + + @POST + @Produces(MediaType.APPLICATION_JSON) + @Consumes(MediaType.APPLICATION_JSON) + public Response doPost( + final SqlQuery sqlQuery, + @Context final HttpServletRequest req + ) + { + final HttpStatement stmt = msqSqlStatementFactory.httpStatement(sqlQuery, req); + final String sqlQueryId = stmt.sqlQueryId(); + final String currThreadName = Thread.currentThread().getName(); + try { + ExecutionMode executionMode = QueryContexts.getAsEnum( + QueryContexts.CTX_EXECUTION_MODE, + sqlQuery.getContext().get(QueryContexts.CTX_EXECUTION_MODE), + ExecutionMode.class + ); + if (ExecutionMode.ASYNC != executionMode) { + return buildNonOkResponse( + HttpStatus.SC_UNPROCESSABLE_ENTITY, + new QueryException( + QueryException.UNSUPPORTED_OPERATION_ERROR_CODE, + StringUtils.format( + "The statement sql api only supports sync mode[%s]. Please set context parameter [%s=%s] in the payload", + ExecutionMode.ASYNC, + QueryContexts.CTX_EXECUTION_MODE, + ExecutionMode.ASYNC + ), + null, + null + ), + stmt.sqlQueryId() + ); + } + + + Thread.currentThread().setName(StringUtils.format("statement_sql[%s]", sqlQueryId)); + + final DirectStatement.ResultSet plan = stmt.plan(); + final QueryResponse response = plan.run(); + final Sequence sequence = response.getResults(); + final SqlRowTransformer rowTransformer = plan.createRowTransformer(); + + final boolean isTaskStruct = MSQTaskSqlEngine.TASK_STRUCT_FIELD_NAMES.equals(rowTransformer.getFieldList()); + + if (isTaskStruct) { + return buildTaskResponse(sequence, stmt.query().authResult().getIdentity()); + } else { + // Used for EXPLAIN + return buildStandardResponse(sequence, sqlQuery, sqlQueryId, rowTransformer); + } + } + // Kitchen-sinking the errors since they are all unchecked. + // Just copied from SqlResource. + catch (QueryCapacityExceededException cap) { + stmt.reporter().failed(cap); + return buildNonOkResponse(QueryCapacityExceededException.STATUS_CODE, cap, sqlQueryId); + } + catch (QueryUnsupportedException unsupported) { + stmt.reporter().failed(unsupported); + return buildNonOkResponse(QueryUnsupportedException.STATUS_CODE, unsupported, sqlQueryId); + } + catch (QueryTimeoutException timeout) { + stmt.reporter().failed(timeout); + return buildNonOkResponse(QueryTimeoutException.STATUS_CODE, timeout, sqlQueryId); + } + catch (SqlPlanningException | ResourceLimitExceededException e) { + stmt.reporter().failed(e); + return buildNonOkResponse(BadQueryException.STATUS_CODE, e, sqlQueryId); + } + catch (ForbiddenException e) { + // No request logs for forbidden queries; same as SqlResource + throw (ForbiddenException) serverConfig.getErrorResponseTransformStrategy() + .transformIfNeeded(e); // let ForbiddenExceptionMapper handle this + } + catch (RelOptPlanner.CannotPlanException e) { + stmt.reporter().failed(e); + SqlPlanningException spe = new SqlPlanningException( + SqlPlanningException.PlanningError.UNSUPPORTED_SQL_ERROR, + e.getMessage() + ); + return buildNonOkResponse(BadQueryException.STATUS_CODE, spe, sqlQueryId); + } + // Calcite throws a java.lang.AssertionError which is type Error not Exception. Using Throwable catches both. + catch (Throwable e) { + stmt.reporter().failed(e); + log.noStackTrace().warn(e, "Failed to handle query: %s", sqlQueryId); + + return buildNonOkResponse( + Response.Status.INTERNAL_SERVER_ERROR.getStatusCode(), + QueryInterruptedException.wrapIfNeeded(e), + sqlQueryId + ); + } + finally { + stmt.close(); + Thread.currentThread().setName(currThreadName); + } + } + + + @GET + @Path("/{id}") + @Produces(MediaType.APPLICATION_JSON) + public Response doGetStatus( + @PathParam("id") final String queryId, + @Context final HttpServletRequest req + ) + { + try { + AuthorizationUtils.authorizeAllResourceActions(req, Collections.emptyList(), authorizerMapper); + final AuthenticationResult authenticationResult = AuthorizationUtils.authenticationResultFromRequest(req); + + Optional sqlStatementResult = getStatementStatus( + queryId, + authenticationResult.getIdentity(), + true + ); + if (sqlStatementResult.isPresent()) { + return Response.ok().entity(sqlStatementResult.get()).build(); + } else { + return Response.status(Response.Status.NOT_FOUND).build(); + } + } + catch (ForbiddenException e) { + throw (ForbiddenException) serverConfig.getErrorResponseTransformStrategy() + .transformIfNeeded(e); + } + catch (QueryException e) { + return buildNonOkResponse(Response.Status.INTERNAL_SERVER_ERROR.getStatusCode(), e, queryId); + } + catch (Exception e) { + log.noStackTrace().warn(e, "Failed to handle query: %s", queryId); + throw e; + } + } + + @GET + @Path("/{id}/results") + @Produces(MediaType.APPLICATION_JSON) + public Response doGetResults( + @PathParam("id") final String queryId, + @QueryParam("offset") Long offset, + @QueryParam("numRows") Long numberOfRows, + @QueryParam("sizeInBytes") Long size, + @QueryParam("timeout") Integer timeout, + @Context final HttpServletRequest req + ) + { + if (offset != null && offset < 0) { + return buildNonOkResponse( + Response.Status.PRECONDITION_FAILED.getStatusCode(), + new QueryException(null, "offset cannot be negative. Please pass a positive number.", null, null), + queryId + ); + } + if (numberOfRows != null && numberOfRows < 0) { + return buildNonOkResponse( + Response.Status.PRECONDITION_FAILED.getStatusCode(), + new QueryException(null, "numRows cannot be negative. Please pass a positive number.", null, null), + queryId + ); + } + + final long start = offset == null ? 0 : offset; + final long last = getLastIndex(numberOfRows, start); + + try { + AuthorizationUtils.authorizeAllResourceActions(req, Collections.emptyList(), authorizerMapper); + final AuthenticationResult authenticationResult = AuthorizationUtils.authenticationResultFromRequest(req); + + TaskStatusResponse taskResponse = overlordClient.getTaskStatus(queryId); + if (taskResponse == null) { + return Response.status(Response.Status.NOT_FOUND).build(); + } + + TaskStatusPlus statusPlus = taskResponse.getStatus(); + if (statusPlus == null || !MSQControllerTask.TYPE.equals(statusPlus.getType())) { + return Response.status(Response.Status.NOT_FOUND).build(); + } + + MSQControllerTask msqControllerTask = getMSQControllerTaskOrThrow(queryId, authenticationResult.getIdentity()); + SqlStatementState sqlStatementState = getSqlStatementState(statusPlus); + + if (sqlStatementState == SqlStatementState.RUNNING || sqlStatementState == SqlStatementState.ACCEPTED) { + return buildNonOkResponse( + Response.Status.PRECONDITION_FAILED.getStatusCode(), + new QueryException(null, "Query not ready to retrieve results", null, null), + queryId + ); + } else if (sqlStatementState == SqlStatementState.FAILED) { + return buildNonOkResponse( + Response.Status.INTERNAL_SERVER_ERROR.getStatusCode(), + new QueryException(null, statusPlus.getErrorMsg(), null, null), + queryId + ); + } else { + Optional> signature = getSignature(msqControllerTask); + Optional> results = getResults(overlordClient.getTaskReport(queryId)); + + return Response.ok( + (StreamingOutput) outputStream -> { + CountingOutputStream os = new CountingOutputStream(outputStream); + + try (final ResultFormat.Writer writer = ResultFormat.OBJECT.createFormatter(os, jsonMapper)) { + List rowSignature = signature.get(); + writer.writeResponseStart(); + + for (int k = (int) start; k < Math.min(last, results.get().size()); k++) { + writer.writeRowStart(); + for (int i = 0; i < rowSignature.size(); i++) { + writer.writeRowField(rowSignature.get(i).getColName(), ((List) results.get().get(k)).get(i)); + } + writer.writeRowEnd(); + } + + writer.writeResponseEnd(); + } + catch (Exception e) { + log.error(e, "Unable to stream results back for query[%s]", queryId); + throw new ISE(e, "Unable to stream results back for query[%s]", queryId); + } + } + ).build(); + + } + } + catch (ForbiddenException e) { + throw (ForbiddenException) serverConfig.getErrorResponseTransformStrategy() + .transformIfNeeded(e); + } + catch (QueryException e) { + return buildNonOkResponse(Response.Status.INTERNAL_SERVER_ERROR.getStatusCode(), e, queryId); + } + catch (Exception e) { + log.noStackTrace().warn(e, "Failed to handle query: %s", queryId); + throw e; + } + } + + /** + * Queries can be canceled while in anystate. Canceling a query that has already completed will remove its results. + * + * @param queryId queryId + * @param req httpServletRequest + * @return HTTP 404 if the query ID does not exist,expired or originated by different user. HTTP 202 if the deletion + * request has been accepted. + */ + @DELETE + @Path("/{id}") + @Produces(MediaType.APPLICATION_JSON) + public Response deleteQuery(@PathParam("id") final String queryId, @Context final HttpServletRequest req) + { + + try { + AuthorizationUtils.authorizeAllResourceActions(req, Collections.emptyList(), authorizerMapper); + final AuthenticationResult authenticationResult = AuthorizationUtils.authenticationResultFromRequest(req); + + Optional sqlStatementResult = getStatementStatus( + queryId, + authenticationResult.getIdentity(), + false + ); + if (sqlStatementResult.isPresent()) { + switch (sqlStatementResult.get().getState()) { + case ACCEPTED: + case RUNNING: + overlordClient.cancelTask(queryId); + return Response.status(Response.Status.ACCEPTED).build(); + case SUCCESS: + case FAILED: + // we would also want to clean up the + return Response.ok().build(); + default: + throw new ISE("Illegal State[%s] encountered", sqlStatementResult.get().getState()); + } + + } else { + return Response.status(Response.Status.NOT_FOUND).build(); + } + + + } + catch (ForbiddenException e) { + throw (ForbiddenException) serverConfig.getErrorResponseTransformStrategy() + .transformIfNeeded(e); + } + catch (QueryException e) { + return buildNonOkResponse(Response.Status.INTERNAL_SERVER_ERROR.getStatusCode(), e, queryId); + } + catch (Exception e) { + log.noStackTrace().warn(e, "Failed to handle query: %s", queryId); + throw e; + } + } + + private Response buildStandardResponse( + Sequence sequence, + SqlQuery sqlQuery, + String sqlQueryId, + SqlRowTransformer rowTransformer + ) throws IOException + { + final Yielder yielder0 = Yielders.each(sequence); + + try { + final Response.ResponseBuilder responseBuilder = Response + .ok( + (StreamingOutput) outputStream -> { + CountingOutputStream os = new CountingOutputStream(outputStream); + Yielder yielder = yielder0; + + try (final ResultFormat.Writer writer = sqlQuery.getResultFormat() + .createFormatter(os, jsonMapper)) { + writer.writeResponseStart(); + + if (sqlQuery.includeHeader()) { + writer.writeHeader( + rowTransformer.getRowType(), + sqlQuery.includeTypesHeader(), + sqlQuery.includeSqlTypesHeader() + ); + } + + while (!yielder.isDone()) { + final Object[] row = yielder.get(); + writer.writeRowStart(); + for (int i = 0; i < rowTransformer.getFieldList().size(); i++) { + final Object value = rowTransformer.transform(row, i); + writer.writeRowField(rowTransformer.getFieldList().get(i), value); + } + writer.writeRowEnd(); + yielder = yielder.next(null); + } + + writer.writeResponseEnd(); + } + catch (Exception e) { + log.error(e, "Unable to send SQL response [%s]", sqlQueryId); + throw new RuntimeException(e); + } + finally { + yielder.close(); + } + } + ); + + if (sqlQuery.includeHeader()) { + responseBuilder.header(SqlResource.SQL_HEADER_RESPONSE_HEADER, SqlResource.SQL_HEADER_VALUE); + } + + return responseBuilder.build(); + } + catch (Throwable e) { + // make sure to close yielder if anything happened before starting to serialize the response. + yielder0.close(); + throw e; + } + } + + private Response buildTaskResponse(Sequence sequence, String user) + { + List rows = sequence.toList(); + int numRows = rows.size(); + if (numRows != 1) { + throw new RE("Expected a single row but got [%d] rows. Please check broker logs for more information.", numRows); + } + String taskId = (String) rows.get(0)[0]; + try { + Optional statementResult = getStatementStatus(taskId, user, true); + + if (statementResult.isPresent()) { + return Response.status(Response.Status.OK).entity(statementResult.get()).build(); + } else { + return buildNonOkResponse( + Response.Status.INTERNAL_SERVER_ERROR.getStatusCode(), + new QueryException( + null, + StringUtils.format( + "Unable to find associated task for query id %s", + taskId + ), + null, + null + ), + taskId + ); + } + } + catch (QueryException e) { + return buildNonOkResponse(Response.Status.OK.getStatusCode(), e, taskId); + } + } + + private Response buildNonOkResponse(int status, SanitizableException e, String sqlQueryId) + { + // Though transformIfNeeded returns an exception, its purpose is to return + // a QueryException + Exception cleaned = serverConfig + .getErrorResponseTransformStrategy() + .transformIfNeeded(e); + return Response + .status(status) + .entity(cleaned) + .build(); + } + + + private static Optional> getSignature( + MSQControllerTask msqControllerTask + ) + { + // only populate signature for select q's + if (msqControllerTask.getQuerySpec().getDestination().getClass() == TaskReportMSQDestination.class) { + ColumnMappings columnMappings = msqControllerTask.getQuerySpec().getColumnMappings(); + List sqlTypeNames = msqControllerTask.getSqlTypeNames(); + if (sqlTypeNames == null) { + return Optional.empty(); + } + List signature = new ArrayList<>(columnMappings.size()); + int index = 0; + for (String colName : columnMappings.getOutputColumnNames()) { + signature.add(new ColNameAndType(colName, sqlTypeNames.get(index).getName())); + index++; + } + return Optional.of(signature); + } + return Optional.empty(); + } + + + private void checkTaskPayloadOrThrow(TaskPayloadResponse taskPayloadResponse, String queryId) throws QueryException + { + if (taskPayloadResponse == null || taskPayloadResponse.getPayload() == null) { + throw new QueryException(QueryException.UNKNOWN_EXCEPTION_ERROR_CODE, + StringUtils.format( + "Could not get payload details of query[%s] from the overlord", + queryId + ), null, null + ); + } + + if (MSQControllerTask.class != taskPayloadResponse.getPayload().getClass()) { + throw new QueryException(QueryException.UNKNOWN_EXCEPTION_ERROR_CODE, + StringUtils.format( + "Fetched an unexpected payload details of query[%s] from the overlord.", + queryId + ), null, null + ); + } + } + + private Optional getSampleResults( + String asyncResultId, + boolean isSelectQuery, + SqlStatementState sqlStatementState + ) + { + // only populate sample results in case a select query is successful + if (isSelectQuery && sqlStatementState == SqlStatementState.SUCCESS) { + Map report = overlordClient.getTaskReport(asyncResultId); + Optional> rows = getResults(report); + + if (rows.isPresent()) { + return Optional.of(new ResultSetInformation( + null, + false, + (long) rows.get().size(), + null, + rows.get() + )); + } else { + return Optional.empty(); + } + + } else { + return Optional.empty(); + } + } + + + private Optional getStatementStatus(String queryId, String currentUser, boolean withResults) + throws QueryException, ForbiddenException + { + TaskStatusResponse taskResponse = overlordClient.getTaskStatus(queryId); + if (taskResponse == null) { + return Optional.empty(); + } + + TaskStatusPlus statusPlus = taskResponse.getStatus(); + if (statusPlus == null || !MSQControllerTask.TYPE.equals(statusPlus.getType())) { + return Optional.empty(); + } + + SqlStatementState sqlStatementState = getSqlStatementState(statusPlus); + + if (SqlStatementState.FAILED == sqlStatementState) { + throw new QueryException(null, statusPlus.getErrorMsg(), null, null); + } + + MSQControllerTask msqControllerTask = getMSQControllerTaskOrThrow(queryId, currentUser); + Optional> signature = getSignature(msqControllerTask); + + return Optional.of(new SqlStatementResult( + queryId, + sqlStatementState, + taskResponse.getStatus() + .getCreatedTime(), + signature.orElse(null), + taskResponse.getStatus().getDuration(), + withResults ? getSampleResults( + queryId, + signature.isPresent(), + sqlStatementState + ).orElse(null) : null + )); + } + + private MSQControllerTask getMSQControllerTaskOrThrow(String queryId, String currentUser) throws ForbiddenException + { + TaskPayloadResponse taskPayloadResponse = overlordClient.getTaskPayload(queryId); + checkTaskPayloadOrThrow(taskPayloadResponse, queryId); + + MSQControllerTask msqControllerTask = (MSQControllerTask) taskPayloadResponse.getPayload(); + if (currentUser == null || !currentUser.equals(msqControllerTask.getQuerySpec() + .getQuery() + .getContext() + .get(MSQTaskQueryMaker.USER_KEY))) { + throw new ForbiddenException(); + } + return msqControllerTask; + } + + private static SqlStatementState getSqlStatementState(TaskStatusPlus taskStatusPlus) + { + TaskState state = taskStatusPlus.getStatusCode(); + if (state == null) { + return SqlStatementState.ACCEPTED; + } + + switch (state) { + case FAILED: + return SqlStatementState.FAILED; + case RUNNING: + if (TaskLocation.unknown().equals(taskStatusPlus.getLocation())) { + return SqlStatementState.ACCEPTED; + } else { + return SqlStatementState.RUNNING; + } + case SUCCESS: + return SqlStatementState.SUCCESS; + default: + throw new ISE("Unrecognized state[%s] found.", state); + } + } + + @SuppressWarnings("unchecked") + private Map getMap(Map map, String key) + { + if (map == null) { + return null; + } + return (Map) map.get(key); + } + + /** + * Get results from report + */ + @SuppressWarnings("unchecked") + private Optional> getResults(Map results) + { + Map msqReport = getMap(results, "multiStageQuery"); + Map payload = getMap(msqReport, "payload"); + Map resultsHolder = getMap(payload, "results"); + + if (resultsHolder == null) { + return Optional.empty(); + } + + List data = (List) resultsHolder.get("results"); + List rows = new ArrayList<>(); + if (data != null) { + rows.addAll(data); + } + return Optional.of(rows); + } + + private static long getLastIndex(Long numberOfRows, long start) + { + final long last; + if (numberOfRows == null) { + last = Long.MAX_VALUE; + } else { + long finalIndex; + try { + finalIndex = Math.addExact(start, numberOfRows); + } + catch (ArithmeticException e) { + finalIndex = Long.MAX_VALUE; + } + last = finalIndex; + } + return last; + } + +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlTaskResource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlTaskResource.java similarity index 98% rename from extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlTaskResource.java rename to extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlTaskResource.java index f0cd7318f644..02130f3e2dbd 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlTaskResource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlTaskResource.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.msq.sql; +package org.apache.druid.msq.sql.resources; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; @@ -31,6 +31,8 @@ import org.apache.druid.java.util.common.guava.Yielder; import org.apache.druid.java.util.common.guava.Yielders; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.msq.sql.MSQTaskSqlEngine; +import org.apache.druid.msq.sql.SqlTaskStatus; import org.apache.druid.query.BadQueryException; import org.apache.druid.query.QueryCapacityExceededException; import org.apache.druid.query.QueryException; diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java index aeca792024a9..69b893b138f0 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/report/MSQTaskReportTest.java @@ -60,7 +60,7 @@ public class MSQTaskReportTest { private static final String TASK_ID = "mytask"; private static final String HOST = "example.com:1234"; - private static final QueryDefinition QUERY_DEFINITION = + public static final QueryDefinition QUERY_DEFINITION = QueryDefinition .builder() .add( diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java new file mode 100644 index 000000000000..f84dc881df31 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java @@ -0,0 +1,490 @@ +/* + * 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.msq.sql; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.api.client.util.Maps; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.druid.client.indexing.IndexingServiceClient; +import org.apache.druid.client.indexing.TaskPayloadResponse; +import org.apache.druid.client.indexing.TaskStatusResponse; +import org.apache.druid.indexer.TaskLocation; +import org.apache.druid.indexer.TaskState; +import org.apache.druid.indexer.TaskStatusPlus; +import org.apache.druid.indexing.common.TaskReport; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.java.util.common.guava.Yielders; +import org.apache.druid.msq.counters.CounterSnapshotsTree; +import org.apache.druid.msq.guice.MSQIndexingModule; +import org.apache.druid.msq.indexing.MSQControllerTask; +import org.apache.druid.msq.indexing.MSQSpec; +import org.apache.druid.msq.indexing.MSQTuningConfig; +import org.apache.druid.msq.indexing.TaskReportMSQDestination; +import org.apache.druid.msq.indexing.report.MSQResultsReport; +import org.apache.druid.msq.indexing.report.MSQStagesReport; +import org.apache.druid.msq.indexing.report.MSQStatusReport; +import org.apache.druid.msq.indexing.report.MSQTaskReport; +import org.apache.druid.msq.indexing.report.MSQTaskReportPayload; +import org.apache.druid.msq.indexing.report.MSQTaskReportTest; +import org.apache.druid.msq.sql.entity.ColNameAndType; +import org.apache.druid.msq.sql.entity.ResultSetInformation; +import org.apache.druid.msq.sql.entity.SqlStatementResult; +import org.apache.druid.msq.sql.resources.SqlStatementResource; +import org.apache.druid.query.Druids; +import org.apache.druid.query.Query; +import org.apache.druid.query.QueryException; +import org.apache.druid.query.scan.ScanQuery; +import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.server.initialization.ServerConfig; +import org.apache.druid.server.mocks.MockHttpServletRequest; +import org.apache.druid.server.security.AuthConfig; +import org.apache.druid.server.security.AuthenticationResult; +import org.apache.druid.sql.SqlStatementFactory; +import org.apache.druid.sql.calcite.planner.ColumnMappings; +import org.apache.druid.sql.calcite.util.CalciteTestBase; +import org.apache.druid.sql.calcite.util.CalciteTests; +import org.apache.druid.sql.http.SqlResourceTest; +import org.joda.time.DateTime; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; +import org.mockito.ArgumentMatchers; +import org.mockito.Mock; +import org.mockito.Mockito; + +import javax.ws.rs.core.Response; +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class SqlStatementResourceTest extends CalciteTestBase +{ + + private static final ObjectMapper JSON_MAPPER = TestHelper.makeJsonMapper(); + private static SqlStatementResource resource; + + private static final String RUNNING_SELECT_MSQ_QUERY = "QUERYID1"; + private static final String FINISHED_SELECT_MSQ_QUERY = "QUERYID2"; + + private static final String ACCEPTED_SELECT_MSQ_QUERY = "QUERYID3"; + + + private static final String ERRORED_SELECT_MSQ_QUERY = "QUERYID4"; + + private static final Query QUERY = + new Druids.ScanQueryBuilder() + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .legacy(false) + .intervals(new MultipleIntervalSegmentSpec( + Collections.singletonList(Intervals.of( + "2011-04-01T00:00:00.000Z/2011-04-03T00:00:00.000Z")))) + .dataSource("target") + .context(ImmutableMap.of(MSQTaskQueryMaker.USER_KEY, AuthConfig.ALLOW_ALL_NAME)) + .build(); + + + private static final MSQControllerTask MSQ_CONTROLLER_TASK_PAYLOAD = new MSQControllerTask( + ACCEPTED_SELECT_MSQ_QUERY, + MSQSpec.builder() + .query(QUERY) + .columnMappings( + ColumnMappings.identity( + RowSignature.builder() + .add( + "_time", + ColumnType.LONG + ) + .add( + "alias", + ColumnType.STRING + ) + .add( + "market", + ColumnType.STRING + ) + .build())) + .destination( + TaskReportMSQDestination.INSTANCE) + .tuningConfig( + MSQTuningConfig.defaultConfig()) + .build(), + "select _time,alias,market from test", + Maps.newHashMap(), + null, + ImmutableList.of(SqlTypeName.TIMESTAMP, SqlTypeName.VARCHAR, SqlTypeName.VARCHAR), + null + ); + + @Mock + private static IndexingServiceClient indexingServiceClient; + + private static final DateTime CREATED_TIME = DateTimes.of("2023-05-31T12:00Z"); + private static final DateTime QUEUE_INSERTION_TIME = DateTimes.of("2023-05-31T12:01Z"); + + private static final List RESULT_ROWS = ImmutableList.of( + new Object[]{123, "foo", "bar"}, + new Object[]{234, "foo1", "bar1"} + ); + + private static final Map ROW1 = ImmutableMap.of("_time", 123, "alias", "foo", "market", "bar"); + private static final Map ROW2 = ImmutableMap.of("_time", 234, "alias", "foo1", "market", "bar1"); + + private static final String FAILURE_MSG = "failure msg"; + + @BeforeClass + public static void init() throws Exception + { + indexingServiceClient = Mockito.mock(IndexingServiceClient.class); + Mockito.when(indexingServiceClient.getTaskStatus(ArgumentMatchers.eq(ACCEPTED_SELECT_MSQ_QUERY))) + .thenReturn(new TaskStatusResponse( + ACCEPTED_SELECT_MSQ_QUERY, + new TaskStatusPlus( + ACCEPTED_SELECT_MSQ_QUERY, + null, + MSQControllerTask.TYPE, + CREATED_TIME, + QUEUE_INSERTION_TIME, + null, + null, + null, + TaskLocation.unknown(), + null, + null + ) + )); + + Mockito.when(indexingServiceClient.getTaskPayload(ArgumentMatchers.eq(ACCEPTED_SELECT_MSQ_QUERY))) + .thenReturn(new TaskPayloadResponse(ACCEPTED_SELECT_MSQ_QUERY, MSQ_CONTROLLER_TASK_PAYLOAD)); + + Mockito.when(indexingServiceClient.getTaskStatus(ArgumentMatchers.eq(RUNNING_SELECT_MSQ_QUERY))) + .thenReturn(new TaskStatusResponse( + RUNNING_SELECT_MSQ_QUERY, + new TaskStatusPlus( + RUNNING_SELECT_MSQ_QUERY, + null, + MSQControllerTask.TYPE, + CREATED_TIME, + QUEUE_INSERTION_TIME, + TaskState.RUNNING, + null, + null, + new TaskLocation("test", 0, 0), + null, + null + ) + )); + + Mockito.when(indexingServiceClient.getTaskPayload(RUNNING_SELECT_MSQ_QUERY)) + .thenReturn(new TaskPayloadResponse(RUNNING_SELECT_MSQ_QUERY, MSQ_CONTROLLER_TASK_PAYLOAD)); + + + Mockito.when(indexingServiceClient.getTaskStatus(ArgumentMatchers.eq(FINISHED_SELECT_MSQ_QUERY))) + .thenReturn(new TaskStatusResponse( + FINISHED_SELECT_MSQ_QUERY, + new TaskStatusPlus( + FINISHED_SELECT_MSQ_QUERY, + null, + MSQControllerTask.TYPE, + CREATED_TIME, + QUEUE_INSERTION_TIME, + TaskState.SUCCESS, + null, + 100L, + new TaskLocation("test", 0, 0), + null, + null + ) + )); + + Mockito.when(indexingServiceClient.getTaskPayload(FINISHED_SELECT_MSQ_QUERY)) + .thenReturn(new TaskPayloadResponse(FINISHED_SELECT_MSQ_QUERY, MSQ_CONTROLLER_TASK_PAYLOAD)); + + + final MSQTaskReport report = new MSQTaskReport( + FINISHED_SELECT_MSQ_QUERY, + new MSQTaskReportPayload( + new MSQStatusReport(TaskState.SUCCESS, null, new ArrayDeque<>(), null, 0, 1, 2), + MSQStagesReport.create( + MSQTaskReportTest.QUERY_DEFINITION, + ImmutableMap.of(), + ImmutableMap.of(), + ImmutableMap.of(), + ImmutableMap.of() + ), + new CounterSnapshotsTree(), + new MSQResultsReport( + ImmutableList.of( + new MSQResultsReport.ColumnAndType("_time", ColumnType.LONG), + new MSQResultsReport.ColumnAndType("alias", ColumnType.STRING), + new MSQResultsReport.ColumnAndType("market", ColumnType.STRING) + ), + ImmutableList.of(SqlTypeName.TIMESTAMP, SqlTypeName.VARCHAR, SqlTypeName.VARCHAR), + Yielders.each(Sequences.simple(RESULT_ROWS)) + ) + ) + ); + + final ObjectMapper mapper = TestHelper.makeJsonMapper() + .registerModules(new MSQIndexingModule().getJacksonModules()); + + Map reportPayload = mapper.readValue( + mapper.writeValueAsString(TaskReport.buildTaskReports(report)), + new TypeReference>() + { + } + ); + + Mockito.when(indexingServiceClient.getTaskReport(FINISHED_SELECT_MSQ_QUERY)) + .thenReturn(reportPayload); + + + Mockito.when(indexingServiceClient.getTaskStatus(ArgumentMatchers.eq(ERRORED_SELECT_MSQ_QUERY))) + .thenReturn(new TaskStatusResponse( + ERRORED_SELECT_MSQ_QUERY, + new TaskStatusPlus( + ERRORED_SELECT_MSQ_QUERY, + null, + MSQControllerTask.TYPE, + CREATED_TIME, + QUEUE_INSERTION_TIME, + TaskState.FAILED, + null, + -1L, + TaskLocation.unknown(), + null, + FAILURE_MSG + ) + )); + + + resource = new SqlStatementResource( + new SqlStatementFactory(null), + new ServerConfig(), + CalciteTests.TEST_AUTHORIZER_MAPPER, + JSON_MAPPER, + indexingServiceClient + ); + } + + + @Test + public void testMSQSelectAcceptedQuery() + { + Response response = resource.doGetStatus(ACCEPTED_SELECT_MSQ_QUERY, makeOkRequest()); + Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); + Assert.assertEquals(new SqlStatementResult( + ACCEPTED_SELECT_MSQ_QUERY, + SqlStatementState.ACCEPTED, + CREATED_TIME, + ImmutableList.of( + new ColNameAndType("_time", SqlTypeName.TIMESTAMP.getName()), + new ColNameAndType("alias", SqlTypeName.VARCHAR.getName()), + new ColNameAndType("market", SqlTypeName.VARCHAR.getName()) + ), + null, + null + ), response.getEntity()); + + Assert.assertEquals( + Response.Status.PRECONDITION_FAILED.getStatusCode(), + resource.doGetResults(ACCEPTED_SELECT_MSQ_QUERY, null, null, null, null, makeOkRequest()) + .getStatus() + ); + Assert.assertEquals( + Response.Status.ACCEPTED.getStatusCode(), + resource.deleteQuery(ACCEPTED_SELECT_MSQ_QUERY, makeOkRequest()).getStatus() + ); + } + + @Test + + public void testMSQSelectRunningQuery() + { + + Response response = resource.doGetStatus(RUNNING_SELECT_MSQ_QUERY, makeOkRequest()); + Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); + Assert.assertEquals(new SqlStatementResult( + RUNNING_SELECT_MSQ_QUERY, + SqlStatementState.RUNNING, + CREATED_TIME, + ImmutableList.of( + new ColNameAndType("_time", SqlTypeName.TIMESTAMP.getName()), + new ColNameAndType("alias", SqlTypeName.VARCHAR.getName()), + new ColNameAndType("market", SqlTypeName.VARCHAR.getName()) + ), + null, + null + ), response.getEntity()); + + Assert.assertEquals( + Response.Status.PRECONDITION_FAILED.getStatusCode(), + resource.doGetResults(RUNNING_SELECT_MSQ_QUERY, null, null, null, null, makeOkRequest()) + .getStatus() + ); + Assert.assertEquals( + Response.Status.ACCEPTED.getStatusCode(), + resource.deleteQuery(RUNNING_SELECT_MSQ_QUERY, makeOkRequest()).getStatus() + ); + } + + + @Test + public void testFinishedSelectMSQQuery() throws Exception + { + Response response = resource.doGetStatus(FINISHED_SELECT_MSQ_QUERY, makeOkRequest()); + Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); + Assert.assertEquals(new SqlStatementResult( + FINISHED_SELECT_MSQ_QUERY, + SqlStatementState.SUCCESS, + CREATED_TIME, + ImmutableList.of( + new ColNameAndType("_time", SqlTypeName.TIMESTAMP.getName()), + new ColNameAndType("alias", SqlTypeName.VARCHAR.getName()), + new ColNameAndType("market", SqlTypeName.VARCHAR.getName()) + ), + 100L, + new ResultSetInformation( + null, + false, + 2L, + null, + RESULT_ROWS.stream().map(Arrays::asList).collect(Collectors.toList()) + ) + ), response.getEntity()); + + Response resultsResponse = resource.doGetResults( + FINISHED_SELECT_MSQ_QUERY, + null, + null, + null, + null, + makeOkRequest() + ); + Assert.assertEquals( + Response.Status.OK.getStatusCode(), resultsResponse.getStatus()); + + List> rows = new ArrayList<>(); + rows.add(ROW1); + rows.add(ROW2); + + Assert.assertEquals(rows, getResultRowsFromResponse(resultsResponse)); + + Assert.assertEquals( + Response.Status.OK.getStatusCode(), + resource.deleteQuery(FINISHED_SELECT_MSQ_QUERY, makeOkRequest()).getStatus() + ); + + Assert.assertEquals( + rows.subList(1, 2), + getResultRowsFromResponse(resource.doGetResults( + FINISHED_SELECT_MSQ_QUERY, + 1L, + null, + null, + null, + makeOkRequest() + )) + ); + Assert.assertEquals( + rows.subList(0, 1), + getResultRowsFromResponse(resource.doGetResults( + FINISHED_SELECT_MSQ_QUERY, + 0L, + 1L, + null, + null, + makeOkRequest() + )) + ); + Assert.assertEquals( + rows, + getResultRowsFromResponse(resource.doGetResults( + FINISHED_SELECT_MSQ_QUERY, + 0L, + 3L, + null, + null, + makeOkRequest() + )) + ); + + Assert.assertEquals( + Response.Status.PRECONDITION_FAILED.getStatusCode(), + resource.doGetResults(FINISHED_SELECT_MSQ_QUERY, -1L, 3L, null, null, makeOkRequest()) + .getStatus() + ); + Assert.assertEquals( + Response.Status.PRECONDITION_FAILED.getStatusCode(), + resource.doGetResults(FINISHED_SELECT_MSQ_QUERY, null, -1L, null, null, makeOkRequest()) + .getStatus() + ); + + } + + + @Test + public void testErroredSelectMSQQuery() + { + Response response = resource.doGetStatus(ERRORED_SELECT_MSQ_QUERY, makeOkRequest()); + Assert.assertEquals(Response.Status.INTERNAL_SERVER_ERROR.getStatusCode(), response.getStatus()); + Assert.assertEquals(FAILURE_MSG, getQueryExceptionFromResponse(response).getMessage()); + } + + private static List getResultRowsFromResponse(Response resultsResponse) throws IOException + { + return JSON_MAPPER.readValue(SqlResourceTest.responseToByteArray(resultsResponse), List.class); + } + + private static QueryException getQueryExceptionFromResponse(Response response) + { + return (QueryException) response.getEntity(); + } + + private MockHttpServletRequest makeOkRequest() + { + return makeExpectedReq(CalciteTests.REGULAR_USER_AUTH_RESULT); + } + + private MockHttpServletRequest makeOtherUserRequest() + { + return makeExpectedReq(CalciteTests.SUPER_USER_AUTH_RESULT); + } + + private MockHttpServletRequest makeExpectedReq(AuthenticationResult authenticationResult) + { + MockHttpServletRequest req = new MockHttpServletRequest(); + req.attributes.put(AuthConfig.DRUID_AUTHENTICATION_RESULT, authenticationResult); + req.remoteAddr = "1.2.3.4"; + return req; + } +} diff --git a/processing/src/main/java/org/apache/druid/query/ExecutionMode.java b/processing/src/main/java/org/apache/druid/query/ExecutionMode.java new file mode 100644 index 000000000000..f17498b29929 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/ExecutionMode.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.query; + +public enum ExecutionMode +{ + + SYNC, + ASYNC + +} diff --git a/processing/src/main/java/org/apache/druid/query/QueryContexts.java b/processing/src/main/java/org/apache/druid/query/QueryContexts.java index f3106d9a6019..4ba4e5d4d919 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryContexts.java +++ b/processing/src/main/java/org/apache/druid/query/QueryContexts.java @@ -86,6 +86,9 @@ public class QueryContexts public static final String CTX_SQL_QUERY_ID = BaseQuery.SQL_QUERY_ID; public static final String CTX_SQL_STRINGIFY_ARRAYS = "sqlStringifyArrays"; + // SQL statement resource specific keys + public static final String CTX_EXECUTION_MODE = "executionMode"; + // Defaults public static final boolean DEFAULT_BY_SEGMENT = false; public static final boolean DEFAULT_POPULATE_CACHE = true; @@ -423,8 +426,20 @@ public static Map override( public static > E getAsEnum(String key, Object value, Class clazz, E defaultValue) { - if (value == null) { + E result = getAsEnum(key, value, clazz); + if (result == null) { return defaultValue; + } else { + return result; + } + } + + + @Nullable + public static > E getAsEnum(String key, Object value, Class clazz) + { + if (value == null) { + return null; } try { diff --git a/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java b/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java index 2a80011a9b01..7a899dd5a913 100644 --- a/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java +++ b/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java @@ -2035,7 +2035,7 @@ private T deserializeResponse(Response resp, Class clazz) throws IOExcept return JSON_MAPPER.readValue(responseToByteArray(resp), clazz); } - private byte[] responseToByteArray(Response resp) throws IOException + public static byte[] responseToByteArray(Response resp) throws IOException { ByteArrayOutputStream baos = new ByteArrayOutputStream(); ((StreamingOutput) resp.getEntity()).write(baos); From 9371ebf1a039a4750b8eddd34427e237532dbfdb Mon Sep 17 00:00:00 2001 From: cryptoe Date: Fri, 16 Jun 2023 08:55:34 +0530 Subject: [PATCH 02/15] Adding more tests. --- .../msq/sql/entity/SqlStatementResult.java | 27 +- .../sql/resources/SqlStatementResource.java | 28 +- .../msq/sql/SqlStatementResourceTest.java | 344 ++++++++++++++++-- .../druid/indexing/common/task/IndexTask.java | 2 +- 4 files changed, 366 insertions(+), 35 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/SqlStatementResult.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/SqlStatementResult.java index afede8cac2a3..3b4e8ea662bc 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/SqlStatementResult.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/SqlStatementResult.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.msq.sql.SqlStatementState; +import org.apache.druid.query.QueryException; import org.joda.time.DateTime; import javax.annotation.Nullable; @@ -47,6 +48,9 @@ public class SqlStatementResult @Nullable private final ResultSetInformation resultSetInformation; + @Nullable + private final QueryException queryException; + @JsonCreator public SqlStatementResult( @@ -61,7 +65,9 @@ public SqlStatementResult( @Nullable @JsonProperty("durationInMs") Long durationInMs, @Nullable @JsonProperty("result") - ResultSetInformation resultSetInformation + ResultSetInformation resultSetInformation, + @Nullable @JsonProperty("exception") + QueryException queryException ) { @@ -71,6 +77,7 @@ public SqlStatementResult( this.sqlRowSignature = sqlRowSignature; this.durationInMs = durationInMs; this.resultSetInformation = resultSetInformation; + this.queryException = queryException; } @JsonProperty @@ -115,6 +122,14 @@ public ResultSetInformation getResultSetInformation() return resultSetInformation; } + @JsonProperty + @Nullable + @JsonInclude(JsonInclude.Include.NON_NULL) + public QueryException getQueryException() + { + return queryException; + } + @Override public boolean equals(Object o) { @@ -131,25 +146,29 @@ public boolean equals(Object o) ) && Objects.equals(sqlRowSignature, that.sqlRowSignature) && Objects.equals( durationInMs, that.durationInMs - ) && Objects.equals(resultSetInformation, that.resultSetInformation); + ) && Objects.equals(resultSetInformation, that.resultSetInformation) && Objects.equals( + queryException, + that.queryException + ); } @Override public int hashCode() { - return Objects.hash(queryId, state, createdAt, sqlRowSignature, durationInMs, resultSetInformation); + return Objects.hash(queryId, state, createdAt, sqlRowSignature, durationInMs, resultSetInformation, queryException); } @Override public String toString() { - return "StatementSqlResult{" + + return "SqlStatementResult{" + "queryId='" + queryId + '\'' + ", state=" + state + ", createdAt=" + createdAt + ", sqlRowSignature=" + sqlRowSignature + ", durationInMs=" + durationInMs + ", resultSetInformation=" + resultSetInformation + + ", queryException=" + queryException + '}'; } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java index a33c43a47d30..58c45781bf83 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java @@ -325,23 +325,27 @@ public Response doGetResults( if (statusPlus == null || !MSQControllerTask.TYPE.equals(statusPlus.getType())) { return Response.status(Response.Status.NOT_FOUND).build(); } - - MSQControllerTask msqControllerTask = getMSQControllerTaskOrThrow(queryId, authenticationResult.getIdentity()); SqlStatementState sqlStatementState = getSqlStatementState(statusPlus); if (sqlStatementState == SqlStatementState.RUNNING || sqlStatementState == SqlStatementState.ACCEPTED) { return buildNonOkResponse( - Response.Status.PRECONDITION_FAILED.getStatusCode(), - new QueryException(null, "Query not ready to retrieve results", null, null), + Response.Status.NOT_FOUND.getStatusCode(), + new QueryException( + null, + StringUtils.format("Query is [%s]. Please wait for it to complete.", sqlStatementState), + null, + null + ), queryId ); } else if (sqlStatementState == SqlStatementState.FAILED) { return buildNonOkResponse( - Response.Status.INTERNAL_SERVER_ERROR.getStatusCode(), + Response.Status.NOT_FOUND.getStatusCode(), new QueryException(null, statusPlus.getErrorMsg(), null, null), queryId ); } else { + MSQControllerTask msqControllerTask = getMSQControllerTaskOrThrow(queryId, authenticationResult.getIdentity()); Optional> signature = getSignature(msqControllerTask); Optional> results = getResults(overlordClient.getTaskReport(queryId)); @@ -642,7 +646,16 @@ private Optional getStatementStatus(String queryId, String c SqlStatementState sqlStatementState = getSqlStatementState(statusPlus); if (SqlStatementState.FAILED == sqlStatementState) { - throw new QueryException(null, statusPlus.getErrorMsg(), null, null); + return Optional.of(new SqlStatementResult( + queryId, + sqlStatementState, + taskResponse.getStatus() + .getCreatedTime(), + null, + taskResponse.getStatus().getDuration(), + null, + new QueryException(null, statusPlus.getErrorMsg(), null, null) + )); } MSQControllerTask msqControllerTask = getMSQControllerTaskOrThrow(queryId, currentUser); @@ -659,7 +672,8 @@ private Optional getStatementStatus(String queryId, String c queryId, signature.isPresent(), sqlStatementState - ).orElse(null) : null + ).orElse(null) : null, + null )); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java index f84dc881df31..c6ad4ae509ab 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java @@ -32,12 +32,16 @@ import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.indexing.common.TaskReport; +import org.apache.druid.indexing.common.task.IndexTask; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.guava.Yielders; import org.apache.druid.msq.counters.CounterSnapshotsTree; import org.apache.druid.msq.guice.MSQIndexingModule; +import org.apache.druid.msq.indexing.DataSourceMSQDestination; import org.apache.druid.msq.indexing.MSQControllerTask; import org.apache.druid.msq.indexing.MSQSpec; import org.apache.druid.msq.indexing.MSQTuningConfig; @@ -93,13 +97,26 @@ public class SqlStatementResourceTest extends CalciteTestBase private static final ObjectMapper JSON_MAPPER = TestHelper.makeJsonMapper(); private static SqlStatementResource resource; - private static final String RUNNING_SELECT_MSQ_QUERY = "QUERYID1"; - private static final String FINISHED_SELECT_MSQ_QUERY = "QUERYID2"; + private static final String ACCEPTED_SELECT_MSQ_QUERY = "QUERY_ID_1"; + private static final String RUNNING_SELECT_MSQ_QUERY = "QUERY_ID_2"; + private static final String FINISHED_SELECT_MSQ_QUERY = "QUERY_ID_3"; - private static final String ACCEPTED_SELECT_MSQ_QUERY = "QUERYID3"; + private static final String ERRORED_SELECT_MSQ_QUERY = "QUERY_ID_4"; - private static final String ERRORED_SELECT_MSQ_QUERY = "QUERYID4"; + private static final String RUNNING_NON_MSQ_TASK = "QUERY_ID_5"; + + private static final String FAILED_NON_MSQ_TASK = "QUERY_ID_6"; + + private static final String FINISHED_NON_MSQ_TASK = "QUERY_ID_7"; + + + private static final String ACCEPTED_INSERT_MSQ_TASK = "QUERY_ID_8"; + + private static final String RUNNING_INSERT_MSQ_QUERY = "QUERY_ID_9"; + private static final String FINISHED_INSERT_MSQ_QUERY = "QUERY_ID_10"; + private static final String ERRORED_INSERT_MSQ_QUERY = "QUERY_ID_11"; + private static final Query QUERY = new Druids.ScanQueryBuilder() @@ -113,7 +130,7 @@ public class SqlStatementResourceTest extends CalciteTestBase .build(); - private static final MSQControllerTask MSQ_CONTROLLER_TASK_PAYLOAD = new MSQControllerTask( + private static final MSQControllerTask MSQ_CONTROLLER_SELECT_PAYLOAD = new MSQControllerTask( ACCEPTED_SELECT_MSQ_QUERY, MSQSpec.builder() .query(QUERY) @@ -145,6 +162,38 @@ public class SqlStatementResourceTest extends CalciteTestBase null ); + private static final MSQControllerTask MSQ_CONTROLLER_INSERT_PAYLOAD = new MSQControllerTask( + ACCEPTED_SELECT_MSQ_QUERY, + MSQSpec.builder() + .query(QUERY) + .columnMappings( + ColumnMappings.identity( + RowSignature.builder() + .add( + "_time", + ColumnType.LONG + ) + .add( + "alias", + ColumnType.STRING + ) + .add( + "market", + ColumnType.STRING + ) + .build())) + .destination( + new DataSourceMSQDestination("test", Granularities.DAY, null, null)) + .tuningConfig( + MSQTuningConfig.defaultConfig()) + .build(), + "insert into test select _time,alias,market from test", + Maps.newHashMap(), + null, + ImmutableList.of(SqlTypeName.TIMESTAMP, SqlTypeName.VARCHAR, SqlTypeName.VARCHAR), + null + ); + @Mock private static IndexingServiceClient indexingServiceClient; @@ -184,7 +233,7 @@ public static void init() throws Exception )); Mockito.when(indexingServiceClient.getTaskPayload(ArgumentMatchers.eq(ACCEPTED_SELECT_MSQ_QUERY))) - .thenReturn(new TaskPayloadResponse(ACCEPTED_SELECT_MSQ_QUERY, MSQ_CONTROLLER_TASK_PAYLOAD)); + .thenReturn(new TaskPayloadResponse(ACCEPTED_SELECT_MSQ_QUERY, MSQ_CONTROLLER_SELECT_PAYLOAD)); Mockito.when(indexingServiceClient.getTaskStatus(ArgumentMatchers.eq(RUNNING_SELECT_MSQ_QUERY))) .thenReturn(new TaskStatusResponse( @@ -205,7 +254,7 @@ public static void init() throws Exception )); Mockito.when(indexingServiceClient.getTaskPayload(RUNNING_SELECT_MSQ_QUERY)) - .thenReturn(new TaskPayloadResponse(RUNNING_SELECT_MSQ_QUERY, MSQ_CONTROLLER_TASK_PAYLOAD)); + .thenReturn(new TaskPayloadResponse(RUNNING_SELECT_MSQ_QUERY, MSQ_CONTROLLER_SELECT_PAYLOAD)); Mockito.when(indexingServiceClient.getTaskStatus(ArgumentMatchers.eq(FINISHED_SELECT_MSQ_QUERY))) @@ -227,8 +276,7 @@ public static void init() throws Exception )); Mockito.when(indexingServiceClient.getTaskPayload(FINISHED_SELECT_MSQ_QUERY)) - .thenReturn(new TaskPayloadResponse(FINISHED_SELECT_MSQ_QUERY, MSQ_CONTROLLER_TASK_PAYLOAD)); - + .thenReturn(new TaskPayloadResponse(FINISHED_SELECT_MSQ_QUERY, MSQ_CONTROLLER_SELECT_PAYLOAD)); final MSQTaskReport report = new MSQTaskReport( FINISHED_SELECT_MSQ_QUERY, @@ -287,6 +335,146 @@ public static void init() throws Exception )); + Mockito.when(indexingServiceClient.getTaskStatus(ArgumentMatchers.eq(RUNNING_NON_MSQ_TASK))) + .thenReturn(new TaskStatusResponse( + RUNNING_NON_MSQ_TASK, + new TaskStatusPlus( + RUNNING_NON_MSQ_TASK, + null, + null, + CREATED_TIME, + QUEUE_INSERTION_TIME, + TaskState.RUNNING, + null, + -1L, + TaskLocation.unknown(), + null, + null + ) + )); + + + Mockito.when(indexingServiceClient.getTaskStatus(ArgumentMatchers.eq(FAILED_NON_MSQ_TASK))) + .thenReturn(new TaskStatusResponse( + FAILED_NON_MSQ_TASK, + new TaskStatusPlus( + FAILED_NON_MSQ_TASK, + null, + null, + CREATED_TIME, + QUEUE_INSERTION_TIME, + TaskState.FAILED, + null, + -1L, + TaskLocation.unknown(), + null, + FAILURE_MSG + ) + )); + + + Mockito.when(indexingServiceClient.getTaskStatus(ArgumentMatchers.eq(FINISHED_NON_MSQ_TASK))) + .thenReturn(new TaskStatusResponse( + FINISHED_NON_MSQ_TASK, + new TaskStatusPlus( + FINISHED_NON_MSQ_TASK, + null, + IndexTask.TYPE, + CREATED_TIME, + QUEUE_INSERTION_TIME, + TaskState.SUCCESS, + null, + -1L, + TaskLocation.unknown(), + null, + null + ) + )); + + + Mockito.when(indexingServiceClient.getTaskStatus(ArgumentMatchers.eq(ACCEPTED_INSERT_MSQ_TASK))) + .thenReturn(new TaskStatusResponse( + ACCEPTED_SELECT_MSQ_QUERY, + new TaskStatusPlus( + ACCEPTED_SELECT_MSQ_QUERY, + null, + MSQControllerTask.TYPE, + CREATED_TIME, + QUEUE_INSERTION_TIME, + null, + null, + null, + TaskLocation.unknown(), + null, + null + ) + )); + + Mockito.when(indexingServiceClient.getTaskPayload(ArgumentMatchers.eq(ACCEPTED_INSERT_MSQ_TASK))) + .thenReturn(new TaskPayloadResponse(ACCEPTED_INSERT_MSQ_TASK, MSQ_CONTROLLER_INSERT_PAYLOAD)); + + Mockito.when(indexingServiceClient.getTaskStatus(ArgumentMatchers.eq(RUNNING_INSERT_MSQ_QUERY))) + .thenReturn(new TaskStatusResponse( + RUNNING_INSERT_MSQ_QUERY, + new TaskStatusPlus( + RUNNING_INSERT_MSQ_QUERY, + null, + MSQControllerTask.TYPE, + CREATED_TIME, + QUEUE_INSERTION_TIME, + TaskState.RUNNING, + null, + null, + new TaskLocation("test", 0, 0), + null, + null + ) + )); + + Mockito.when(indexingServiceClient.getTaskPayload(RUNNING_INSERT_MSQ_QUERY)) + .thenReturn(new TaskPayloadResponse(RUNNING_INSERT_MSQ_QUERY, MSQ_CONTROLLER_INSERT_PAYLOAD)); + + + Mockito.when(indexingServiceClient.getTaskStatus(ArgumentMatchers.eq(FINISHED_INSERT_MSQ_QUERY))) + .thenReturn(new TaskStatusResponse( + FINISHED_INSERT_MSQ_QUERY, + new TaskStatusPlus( + FINISHED_INSERT_MSQ_QUERY, + null, + MSQControllerTask.TYPE, + CREATED_TIME, + QUEUE_INSERTION_TIME, + TaskState.SUCCESS, + null, + 100L, + new TaskLocation("test", 0, 0), + null, + null + ) + )); + + Mockito.when(indexingServiceClient.getTaskPayload(FINISHED_INSERT_MSQ_QUERY)) + .thenReturn(new TaskPayloadResponse(FINISHED_INSERT_MSQ_QUERY, MSQ_CONTROLLER_INSERT_PAYLOAD)); + + + Mockito.when(indexingServiceClient.getTaskStatus(ArgumentMatchers.eq(ERRORED_INSERT_MSQ_QUERY))) + .thenReturn(new TaskStatusResponse( + ERRORED_INSERT_MSQ_QUERY, + new TaskStatusPlus( + ERRORED_INSERT_MSQ_QUERY, + null, + MSQControllerTask.TYPE, + CREATED_TIME, + QUEUE_INSERTION_TIME, + TaskState.FAILED, + null, + -1L, + TaskLocation.unknown(), + null, + FAILURE_MSG + ) + )); + resource = new SqlStatementResource( new SqlStatementFactory(null), new ServerConfig(), @@ -312,13 +500,14 @@ public void testMSQSelectAcceptedQuery() new ColNameAndType("market", SqlTypeName.VARCHAR.getName()) ), null, + null, null ), response.getEntity()); - Assert.assertEquals( - Response.Status.PRECONDITION_FAILED.getStatusCode(), - resource.doGetResults(ACCEPTED_SELECT_MSQ_QUERY, null, null, null, null, makeOkRequest()) - .getStatus() + assertExceptionMessage( + resource.doGetResults(ACCEPTED_SELECT_MSQ_QUERY, null, null, null, null, makeOkRequest()), + StringUtils.format("Query is [%s]. Please wait for it to complete.", SqlStatementState.ACCEPTED), + Response.Status.NOT_FOUND ); Assert.assertEquals( Response.Status.ACCEPTED.getStatusCode(), @@ -343,13 +532,14 @@ public void testMSQSelectRunningQuery() new ColNameAndType("market", SqlTypeName.VARCHAR.getName()) ), null, + null, null ), response.getEntity()); - Assert.assertEquals( - Response.Status.PRECONDITION_FAILED.getStatusCode(), - resource.doGetResults(RUNNING_SELECT_MSQ_QUERY, null, null, null, null, makeOkRequest()) - .getStatus() + assertExceptionMessage( + resource.doGetResults(RUNNING_SELECT_MSQ_QUERY, null, null, null, null, makeOkRequest()), + StringUtils.format("Query is [%s]. Please wait for it to complete.", SqlStatementState.RUNNING), + Response.Status.NOT_FOUND ); Assert.assertEquals( Response.Status.ACCEPTED.getStatusCode(), @@ -379,7 +569,8 @@ public void testFinishedSelectMSQQuery() throws Exception 2L, null, RESULT_ROWS.stream().map(Arrays::asList).collect(Collectors.toList()) - ) + ), + null ), response.getEntity()); Response resultsResponse = resource.doGetResults( @@ -453,11 +644,114 @@ public void testFinishedSelectMSQQuery() throws Exception @Test - public void testErroredSelectMSQQuery() + public void testFailedMSQQuery() + { + for (String queryID : ImmutableList.of(ERRORED_SELECT_MSQ_QUERY, ERRORED_INSERT_MSQ_QUERY)) { + assertExceptionMessage( + resource.doGetStatus(queryID, makeOkRequest()), + FAILURE_MSG, + Response.Status.OK + ); + assertExceptionMessage( + resource.doGetResults(queryID, null, null, null, null, makeOkRequest()), + FAILURE_MSG, + Response.Status.NOT_FOUND + ); + + Assert.assertEquals( + Response.Status.OK.getStatusCode(), + resource.deleteQuery(queryID, makeOkRequest()).getStatus() + ); + } + } + + + @Test + public void testNonMSQTasks() + { + for (String queryID : ImmutableList.of(RUNNING_NON_MSQ_TASK, FAILED_NON_MSQ_TASK, FINISHED_NON_MSQ_TASK)) { + assertNullResponse( + resource.doGetStatus(queryID, makeOkRequest()), + Response.Status.NOT_FOUND + ); + assertNullResponse( + resource.doGetResults(queryID, null, null, null, null, makeOkRequest()), + Response.Status.NOT_FOUND + ); + assertNullResponse( + resource.deleteQuery(queryID, makeOkRequest()), + Response.Status.NOT_FOUND + ); + } + } + + @Test + public void testMSQInsertAcceptedQuery() + { + Response response = resource.doGetStatus(ACCEPTED_INSERT_MSQ_TASK, makeOkRequest()); + Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); + Assert.assertEquals(new SqlStatementResult( + ACCEPTED_INSERT_MSQ_TASK, + SqlStatementState.ACCEPTED, + CREATED_TIME, + null, + null, + null, + null + ), response.getEntity()); + + assertExceptionMessage( + resource.doGetResults(ACCEPTED_INSERT_MSQ_TASK, null, null, null, null, makeOkRequest()), + StringUtils.format("Query is [%s]. Please wait for it to complete.", SqlStatementState.ACCEPTED), + Response.Status.NOT_FOUND + ); + Assert.assertEquals( + Response.Status.ACCEPTED.getStatusCode(), + resource.deleteQuery(ACCEPTED_INSERT_MSQ_TASK, makeOkRequest()).getStatus() + ); + } + + @Test + public void testMSQInsertRunningQuery() + { + Response response = resource.doGetStatus(RUNNING_INSERT_MSQ_QUERY, makeOkRequest()); + Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); + Assert.assertEquals(new SqlStatementResult( + RUNNING_INSERT_MSQ_QUERY, + SqlStatementState.RUNNING, + CREATED_TIME, + null, + null, + null, + null + ), response.getEntity()); + + assertExceptionMessage( + resource.doGetResults(RUNNING_INSERT_MSQ_QUERY, null, null, null, null, makeOkRequest()), + StringUtils.format("Query is [%s]. Please wait for it to complete.", SqlStatementState.RUNNING), + Response.Status.NOT_FOUND + ); + Assert.assertEquals( + Response.Status.ACCEPTED.getStatusCode(), + resource.deleteQuery(RUNNING_INSERT_MSQ_QUERY, makeOkRequest()).getStatus() + ); + } + + + private static void assertNullResponse(Response response, Response.Status expectectedStatus) + { + Assert.assertEquals(expectectedStatus.getStatusCode(), response.getStatus()); + Assert.assertNull(response.getEntity()); + } + + private static void assertExceptionMessage( + Response response, + String exceptionMessage, + Response.Status expectectedStatus + ) { - Response response = resource.doGetStatus(ERRORED_SELECT_MSQ_QUERY, makeOkRequest()); - Assert.assertEquals(Response.Status.INTERNAL_SERVER_ERROR.getStatusCode(), response.getStatus()); - Assert.assertEquals(FAILURE_MSG, getQueryExceptionFromResponse(response).getMessage()); + Assert.assertEquals(expectectedStatus.getStatusCode(), response.getStatus()); + Assert.assertEquals(exceptionMessage, getQueryExceptionFromResponse(response).getMessage()); } private static List getResultRowsFromResponse(Response resultsResponse) throws IOException @@ -467,7 +761,11 @@ private static List getResultRowsFromResponse(Response resultsResponse) throws I private static QueryException getQueryExceptionFromResponse(Response response) { - return (QueryException) response.getEntity(); + if (response.getEntity() instanceof SqlStatementResult) { + return ((SqlStatementResult) response.getEntity()).getQueryException(); + } else { + return (QueryException) response.getEntity(); + } } private MockHttpServletRequest makeOkRequest() diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index 5e3d7b65fbfb..9fbf8a6b50b4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java @@ -144,8 +144,8 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler public static final HashFunction HASH_FUNCTION = Hashing.murmur3_128(); + public static final String TYPE = "index"; private static final Logger log = new Logger(IndexTask.class); - private static final String TYPE = "index"; private static String makeGroupId(IndexIngestionSpec ingestionSchema, IngestionMode ingestionMode) { From ed59f6d70ffe9c984725b995b840ff12badafd63 Mon Sep 17 00:00:00 2001 From: cryptoe Date: Tue, 20 Jun 2023 08:26:26 +0530 Subject: [PATCH 03/15] Moving towards the new overlord client. Adding more sanity post test --- .../druid/msq/sql/MSQTaskSqlEngine.java | 1 - .../msq/sql/entity/ResultSetInformation.java | 15 +- .../msq/sql/entity/SqlStatementResult.java | 6 +- .../sql/resources/SqlStatementResource.java | 32 +- .../sql/SqlMsqStatementResourcePostTest.java | 129 +++++++ .../msq/sql/SqlStatementResourceTest.java | 337 ++++++++++++------ .../apache/druid/msq/test/MSQTestBase.java | 11 +- .../test/MSQTestOverlordServiceClient.java | 80 ++++- .../druid/rpc/indexing/OverlordClient.java | 3 + .../rpc/indexing/OverlordClientImpl.java | 17 + .../client/indexing/NoopOverlordClient.java | 6 + .../druid/sql/http/SqlResourceTest.java | 4 + 12 files changed, 483 insertions(+), 158 deletions(-) create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskSqlEngine.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskSqlEngine.java index 0788e69929bf..d05f29d68e59 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskSqlEngine.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskSqlEngine.java @@ -297,7 +297,6 @@ private static RelDataType getMSQStructType(RelDataTypeFactory typeFactory) { return typeFactory.createStructType( ImmutableList.of( - Calcites.createSqlType(typeFactory, SqlTypeName.VARCHAR), Calcites.createSqlType(typeFactory, SqlTypeName.VARCHAR) ), TASK_STRUCT_FIELD_NAMES diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ResultSetInformation.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ResultSetInformation.java index d4533e1c9daa..2eceb6b8796d 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ResultSetInformation.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ResultSetInformation.java @@ -38,7 +38,6 @@ public class ResultSetInformation private final ResultFormat resultFormat; - private final Boolean header; @Nullable private final List records; @@ -47,8 +46,6 @@ public class ResultSetInformation public ResultSetInformation( @Nullable @JsonProperty ResultFormat resultFormat, - @Nullable - @JsonProperty Boolean header, @JsonProperty @Nullable Long totalRows, @JsonProperty @Nullable Long totalSize, @JsonProperty("sampleRecords") @Nullable @@ -58,7 +55,6 @@ public ResultSetInformation( this.totalRows = totalRows; this.totalSize = totalSize; this.resultFormat = resultFormat; - this.header = header; this.records = records; } @@ -86,13 +82,6 @@ public ResultFormat getResultFormat() return resultFormat; } - @JsonProperty - @Nullable - @JsonInclude(JsonInclude.Include.NON_NULL) - public Boolean getHeader() - { - return header; - } @Nullable @JsonProperty("sampleRecords") @@ -116,14 +105,13 @@ public boolean equals(Object o) return Objects.equals(totalRows, that.totalRows) && Objects.equals(totalSize, that.totalSize) && resultFormat == that.resultFormat - && Objects.equals(header, that.header) && Objects.equals(records, that.records); } @Override public int hashCode() { - return Objects.hash(totalRows, totalSize, resultFormat, header, records); + return Objects.hash(totalRows, totalSize, resultFormat, records); } @Override @@ -133,7 +121,6 @@ public String toString() "totalRows=" + totalRows + ", totalSize=" + totalSize + ", resultFormat=" + resultFormat + - ", header=" + header + ", records=" + records + '}'; } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/SqlStatementResult.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/SqlStatementResult.java index 3b4e8ea662bc..1d4c6c9d726b 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/SqlStatementResult.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/SqlStatementResult.java @@ -62,8 +62,8 @@ public SqlStatementResult( DateTime createdAt, @Nullable @JsonProperty("schema") List sqlRowSignature, - @Nullable @JsonProperty("durationInMs") - Long durationInMs, + @Nullable @JsonProperty("durationMs") + Long durationMs, @Nullable @JsonProperty("result") ResultSetInformation resultSetInformation, @Nullable @JsonProperty("exception") @@ -75,7 +75,7 @@ public SqlStatementResult( this.state = state; this.createdAt = createdAt; this.sqlRowSignature = sqlRowSignature; - this.durationInMs = durationInMs; + this.durationInMs = durationMs; this.resultSetInformation = resultSetInformation; this.queryException = queryException; } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java index 58c45781bf83..eb3f70fd20b3 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java @@ -23,13 +23,14 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import com.google.common.io.CountingOutputStream; +import com.google.common.util.concurrent.ListenableFuture; import com.google.inject.Inject; import org.apache.calcite.plan.RelOptPlanner; import org.apache.calcite.sql.type.SqlTypeName; -import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.client.indexing.TaskPayloadResponse; import org.apache.druid.client.indexing.TaskStatusResponse; import org.apache.druid.common.exception.SanitizableException; +import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.guice.annotations.MSQ; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskState; @@ -58,6 +59,7 @@ import org.apache.druid.query.QueryTimeoutException; import org.apache.druid.query.QueryUnsupportedException; import org.apache.druid.query.ResourceLimitExceededException; +import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.server.QueryResponse; import org.apache.druid.server.initialization.ServerConfig; import org.apache.druid.server.security.Access; @@ -106,7 +108,7 @@ public class SqlStatementResource private final ServerConfig serverConfig; private final AuthorizerMapper authorizerMapper; private final ObjectMapper jsonMapper; - private final IndexingServiceClient overlordClient; + private final OverlordClient overlordClient; @Inject @@ -115,7 +117,7 @@ public SqlStatementResource( final ServerConfig serverConfig, final AuthorizerMapper authorizerMapper, final ObjectMapper jsonMapper, - final IndexingServiceClient overlordClient + final OverlordClient overlordClient ) { this.msqSqlStatementFactory = msqSqlStatementFactory; @@ -316,7 +318,7 @@ public Response doGetResults( AuthorizationUtils.authorizeAllResourceActions(req, Collections.emptyList(), authorizerMapper); final AuthenticationResult authenticationResult = AuthorizationUtils.authenticationResultFromRequest(req); - TaskStatusResponse taskResponse = overlordClient.getTaskStatus(queryId); + TaskStatusResponse taskResponse = overlordWork(overlordClient.taskStatus(queryId)); if (taskResponse == null) { return Response.status(Response.Status.NOT_FOUND).build(); } @@ -347,7 +349,10 @@ public Response doGetResults( } else { MSQControllerTask msqControllerTask = getMSQControllerTaskOrThrow(queryId, authenticationResult.getIdentity()); Optional> signature = getSignature(msqControllerTask); - Optional> results = getResults(overlordClient.getTaskReport(queryId)); + if (!signature.isPresent()) { + return Response.ok().build(); + } + Optional> results = getResults(overlordWork(overlordClient.taskReportAsMap(queryId))); return Response.ok( (StreamingOutput) outputStream -> { @@ -609,13 +614,12 @@ private Optional getSampleResults( { // only populate sample results in case a select query is successful if (isSelectQuery && sqlStatementState == SqlStatementState.SUCCESS) { - Map report = overlordClient.getTaskReport(asyncResultId); + Map report = overlordWork(overlordClient.taskReportAsMap(asyncResultId)); Optional> rows = getResults(report); if (rows.isPresent()) { return Optional.of(new ResultSetInformation( null, - false, (long) rows.get().size(), null, rows.get() @@ -633,7 +637,7 @@ private Optional getSampleResults( private Optional getStatementStatus(String queryId, String currentUser, boolean withResults) throws QueryException, ForbiddenException { - TaskStatusResponse taskResponse = overlordClient.getTaskStatus(queryId); + TaskStatusResponse taskResponse = overlordWork(overlordClient.taskStatus(queryId)); if (taskResponse == null) { return Optional.empty(); } @@ -679,7 +683,7 @@ private Optional getStatementStatus(String queryId, String c private MSQControllerTask getMSQControllerTaskOrThrow(String queryId, String currentUser) throws ForbiddenException { - TaskPayloadResponse taskPayloadResponse = overlordClient.getTaskPayload(queryId); + TaskPayloadResponse taskPayloadResponse = overlordWork(overlordClient.taskPayload(queryId)); checkTaskPayloadOrThrow(taskPayloadResponse, queryId); MSQControllerTask msqControllerTask = (MSQControllerTask) taskPayloadResponse.getPayload(); @@ -692,6 +696,16 @@ private MSQControllerTask getMSQControllerTaskOrThrow(String queryId, String cur return msqControllerTask; } + private T overlordWork(final ListenableFuture future) + { + try { + return FutureUtils.getUnchecked(future, true); + } + catch (RuntimeException e) { + throw new QueryException(null, "Unable to contact overlord " + e.getMessage(), null, null); + } + } + private static SqlStatementState getSqlStatementState(TaskStatusPlus taskStatusPlus) { TaskState state = taskStatusPlus.getStatusCode(); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java new file mode 100644 index 000000000000..99390da07dcf --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java @@ -0,0 +1,129 @@ +/* + * 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.msq.sql; + + +import com.fasterxml.jackson.core.type.TypeReference; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.druid.msq.sql.entity.ColNameAndType; +import org.apache.druid.msq.sql.entity.ResultSetInformation; +import org.apache.druid.msq.sql.entity.SqlStatementResult; +import org.apache.druid.msq.sql.resources.SqlStatementResource; +import org.apache.druid.msq.test.MSQTestBase; +import org.apache.druid.msq.test.MSQTestOverlordServiceClient; +import org.apache.druid.query.ExecutionMode; +import org.apache.druid.query.QueryContexts; +import org.apache.druid.server.initialization.ServerConfig; +import org.apache.druid.sql.calcite.util.CalciteTests; +import org.apache.druid.sql.http.SqlQuery; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import javax.ws.rs.core.Response; +import java.io.IOException; +import java.util.List; + +public class SqlMsqStatementResourcePostTest extends MSQTestBase +{ + + private SqlStatementResource resource; + + @Before + public void init() + { + resource = new SqlStatementResource( + sqlStatementFactory, + new ServerConfig(), + CalciteTests.TEST_AUTHORIZER_MAPPER, + objectMapper, + indexingServiceClient + ); + } + + @Test + public void testMSQSelectQueryTest() throws IOException + { + List results = ImmutableList.of( + new Object[]{1L, ""}, + new Object[]{ + 1L, + "10.1" + }, + new Object[]{1L, "2"}, + new Object[]{1L, "1"}, + new Object[]{1L, "def"}, + new Object[]{1L, "abc"} + ); + + Response response = resource.doPost(new SqlQuery( + "select cnt,dim1 from foo", + null, + false, + false, + false, + ImmutableMap.of( + QueryContexts.CTX_EXECUTION_MODE, ExecutionMode.ASYNC.name()), + null + ), SqlStatementResourceTest.makeOkRequest()); + + + Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); + + String taskId = ((SqlStatementResult) response.getEntity()).getQueryId(); + + SqlStatementResult sqlStatementResult = + new SqlStatementResult(taskId, SqlStatementState.SUCCESS, + MSQTestOverlordServiceClient.CREATED_TIME, + ImmutableList.of( + new ColNameAndType( + "cnt", + SqlTypeName.BIGINT.getName() + ), + new ColNameAndType( + "dim1", + SqlTypeName.VARCHAR.getName() + ) + ), + MSQTestOverlordServiceClient.DURATION, + new ResultSetInformation( + null, + 6L, + null, + objectMapper.readValue( + objectMapper.writeValueAsString( + results), + new TypeReference>() + { + } + ) + ), + null + ); + + Assert.assertEquals(sqlStatementResult, response.getEntity()); + + + } + + +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java index c6ad4ae509ab..08d3652e15df 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java @@ -19,13 +19,14 @@ package org.apache.druid.msq.sql; +import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.api.client.util.Maps; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.util.concurrent.Futures; import org.apache.calcite.sql.type.SqlTypeName; -import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.client.indexing.TaskPayloadResponse; import org.apache.druid.client.indexing.TaskStatusResponse; import org.apache.druid.indexer.TaskLocation; @@ -56,11 +57,13 @@ import org.apache.druid.msq.sql.entity.ResultSetInformation; import org.apache.druid.msq.sql.entity.SqlStatementResult; import org.apache.druid.msq.sql.resources.SqlStatementResource; +import org.apache.druid.msq.test.MSQTestBase; import org.apache.druid.query.Druids; import org.apache.druid.query.Query; import org.apache.druid.query.QueryException; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; +import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -68,14 +71,12 @@ import org.apache.druid.server.mocks.MockHttpServletRequest; import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthenticationResult; -import org.apache.druid.sql.SqlStatementFactory; import org.apache.druid.sql.calcite.planner.ColumnMappings; -import org.apache.druid.sql.calcite.util.CalciteTestBase; import org.apache.druid.sql.calcite.util.CalciteTests; import org.apache.druid.sql.http.SqlResourceTest; import org.joda.time.DateTime; import org.junit.Assert; -import org.junit.BeforeClass; +import org.junit.Before; import org.junit.Test; import org.mockito.ArgumentMatchers; import org.mockito.Mock; @@ -91,7 +92,7 @@ import java.util.Map; import java.util.stream.Collectors; -public class SqlStatementResourceTest extends CalciteTestBase +public class SqlStatementResourceTest extends MSQTestBase { private static final ObjectMapper JSON_MAPPER = TestHelper.makeJsonMapper(); @@ -194,28 +195,82 @@ public class SqlStatementResourceTest extends CalciteTestBase null ); + private static final List RESULT_ROWS = ImmutableList.of( + new Object[]{123, "foo", "bar"}, + new Object[]{234, "foo1", "bar1"} + ); + + private static final MSQTaskReport MSQ_SELECT_TASK_REPORT = new MSQTaskReport( + FINISHED_SELECT_MSQ_QUERY, + new MSQTaskReportPayload( + new MSQStatusReport(TaskState.SUCCESS, null, new ArrayDeque<>(), null, 0, 1, 2), + MSQStagesReport.create( + MSQTaskReportTest.QUERY_DEFINITION, + ImmutableMap.of(), + ImmutableMap.of(), + ImmutableMap.of(), + ImmutableMap.of() + ), + new CounterSnapshotsTree(), + new MSQResultsReport( + ImmutableList.of( + new MSQResultsReport.ColumnAndType("_time", ColumnType.LONG), + new MSQResultsReport.ColumnAndType("alias", ColumnType.STRING), + new MSQResultsReport.ColumnAndType("market", ColumnType.STRING) + ), + ImmutableList.of(SqlTypeName.TIMESTAMP, SqlTypeName.VARCHAR, SqlTypeName.VARCHAR), + Yielders.each(Sequences.simple(RESULT_ROWS)) + ) + ) + ); + + private static final MSQTaskReport MSQ_INSERT_TASK_REPORT = new MSQTaskReport( + FINISHED_INSERT_MSQ_QUERY, + new MSQTaskReportPayload( + new MSQStatusReport(TaskState.SUCCESS, null, new ArrayDeque<>(), null, 0, 1, 2), + MSQStagesReport.create( + MSQTaskReportTest.QUERY_DEFINITION, + ImmutableMap.of(), + ImmutableMap.of(), + ImmutableMap.of(), + ImmutableMap.of() + ), + new CounterSnapshotsTree(), + null + ) + ); + @Mock - private static IndexingServiceClient indexingServiceClient; + private static OverlordClient overlordClient; private static final DateTime CREATED_TIME = DateTimes.of("2023-05-31T12:00Z"); private static final DateTime QUEUE_INSERTION_TIME = DateTimes.of("2023-05-31T12:01Z"); - private static final List RESULT_ROWS = ImmutableList.of( - new Object[]{123, "foo", "bar"}, - new Object[]{234, "foo1", "bar1"} - ); private static final Map ROW1 = ImmutableMap.of("_time", 123, "alias", "foo", "market", "bar"); private static final Map ROW2 = ImmutableMap.of("_time", 234, "alias", "foo1", "market", "bar1"); private static final String FAILURE_MSG = "failure msg"; - @BeforeClass - public static void init() throws Exception + @Before + public void init() throws Exception { - indexingServiceClient = Mockito.mock(IndexingServiceClient.class); - Mockito.when(indexingServiceClient.getTaskStatus(ArgumentMatchers.eq(ACCEPTED_SELECT_MSQ_QUERY))) - .thenReturn(new TaskStatusResponse( + overlordClient = Mockito.mock(OverlordClient.class); + setupMocks(overlordClient); + resource = new SqlStatementResource( + sqlStatementFactory, + new ServerConfig(), + CalciteTests.TEST_AUTHORIZER_MAPPER, + JSON_MAPPER, + overlordClient + ); + } + + private static void setupMocks(OverlordClient indexingServiceClient) throws JsonProcessingException + { + + Mockito.when(indexingServiceClient.taskStatus(ArgumentMatchers.eq(ACCEPTED_SELECT_MSQ_QUERY))) + .thenReturn(Futures.immediateFuture(new TaskStatusResponse( ACCEPTED_SELECT_MSQ_QUERY, new TaskStatusPlus( ACCEPTED_SELECT_MSQ_QUERY, @@ -230,13 +285,16 @@ public static void init() throws Exception null, null ) - )); + ))); - Mockito.when(indexingServiceClient.getTaskPayload(ArgumentMatchers.eq(ACCEPTED_SELECT_MSQ_QUERY))) - .thenReturn(new TaskPayloadResponse(ACCEPTED_SELECT_MSQ_QUERY, MSQ_CONTROLLER_SELECT_PAYLOAD)); + Mockito.when(indexingServiceClient.taskPayload(ArgumentMatchers.eq(ACCEPTED_SELECT_MSQ_QUERY))) + .thenReturn(Futures.immediateFuture(new TaskPayloadResponse( + ACCEPTED_SELECT_MSQ_QUERY, + MSQ_CONTROLLER_SELECT_PAYLOAD + ))); - Mockito.when(indexingServiceClient.getTaskStatus(ArgumentMatchers.eq(RUNNING_SELECT_MSQ_QUERY))) - .thenReturn(new TaskStatusResponse( + Mockito.when(indexingServiceClient.taskStatus(ArgumentMatchers.eq(RUNNING_SELECT_MSQ_QUERY))) + .thenReturn(Futures.immediateFuture(new TaskStatusResponse( RUNNING_SELECT_MSQ_QUERY, new TaskStatusPlus( RUNNING_SELECT_MSQ_QUERY, @@ -251,14 +309,17 @@ public static void init() throws Exception null, null ) - )); + ))); - Mockito.when(indexingServiceClient.getTaskPayload(RUNNING_SELECT_MSQ_QUERY)) - .thenReturn(new TaskPayloadResponse(RUNNING_SELECT_MSQ_QUERY, MSQ_CONTROLLER_SELECT_PAYLOAD)); + Mockito.when(indexingServiceClient.taskPayload(RUNNING_SELECT_MSQ_QUERY)) + .thenReturn(Futures.immediateFuture(new TaskPayloadResponse( + RUNNING_SELECT_MSQ_QUERY, + MSQ_CONTROLLER_SELECT_PAYLOAD + ))); - Mockito.when(indexingServiceClient.getTaskStatus(ArgumentMatchers.eq(FINISHED_SELECT_MSQ_QUERY))) - .thenReturn(new TaskStatusResponse( + Mockito.when(indexingServiceClient.taskStatus(ArgumentMatchers.eq(FINISHED_SELECT_MSQ_QUERY))) + .thenReturn(Futures.immediateFuture(new TaskStatusResponse( FINISHED_SELECT_MSQ_QUERY, new TaskStatusPlus( FINISHED_SELECT_MSQ_QUERY, @@ -273,51 +334,30 @@ public static void init() throws Exception null, null ) - )); + ))); - Mockito.when(indexingServiceClient.getTaskPayload(FINISHED_SELECT_MSQ_QUERY)) - .thenReturn(new TaskPayloadResponse(FINISHED_SELECT_MSQ_QUERY, MSQ_CONTROLLER_SELECT_PAYLOAD)); + Mockito.when(indexingServiceClient.taskPayload(FINISHED_SELECT_MSQ_QUERY)) + .thenReturn(Futures.immediateFuture(new TaskPayloadResponse( + FINISHED_SELECT_MSQ_QUERY, + MSQ_CONTROLLER_SELECT_PAYLOAD + ))); - final MSQTaskReport report = new MSQTaskReport( - FINISHED_SELECT_MSQ_QUERY, - new MSQTaskReportPayload( - new MSQStatusReport(TaskState.SUCCESS, null, new ArrayDeque<>(), null, 0, 1, 2), - MSQStagesReport.create( - MSQTaskReportTest.QUERY_DEFINITION, - ImmutableMap.of(), - ImmutableMap.of(), - ImmutableMap.of(), - ImmutableMap.of() - ), - new CounterSnapshotsTree(), - new MSQResultsReport( - ImmutableList.of( - new MSQResultsReport.ColumnAndType("_time", ColumnType.LONG), - new MSQResultsReport.ColumnAndType("alias", ColumnType.STRING), - new MSQResultsReport.ColumnAndType("market", ColumnType.STRING) - ), - ImmutableList.of(SqlTypeName.TIMESTAMP, SqlTypeName.VARCHAR, SqlTypeName.VARCHAR), - Yielders.each(Sequences.simple(RESULT_ROWS)) - ) - ) - ); final ObjectMapper mapper = TestHelper.makeJsonMapper() .registerModules(new MSQIndexingModule().getJacksonModules()); - Map reportPayload = mapper.readValue( - mapper.writeValueAsString(TaskReport.buildTaskReports(report)), - new TypeReference>() - { - } - ); - Mockito.when(indexingServiceClient.getTaskReport(FINISHED_SELECT_MSQ_QUERY)) - .thenReturn(reportPayload); + Mockito.when(indexingServiceClient.taskReportAsMap(FINISHED_SELECT_MSQ_QUERY)) + .thenReturn(Futures.immediateFuture(mapper.readValue( + mapper.writeValueAsString(TaskReport.buildTaskReports(MSQ_SELECT_TASK_REPORT)), + new TypeReference>() + { + } + ))); - Mockito.when(indexingServiceClient.getTaskStatus(ArgumentMatchers.eq(ERRORED_SELECT_MSQ_QUERY))) - .thenReturn(new TaskStatusResponse( + Mockito.when(indexingServiceClient.taskStatus(ArgumentMatchers.eq(ERRORED_SELECT_MSQ_QUERY))) + .thenReturn(Futures.immediateFuture(new TaskStatusResponse( ERRORED_SELECT_MSQ_QUERY, new TaskStatusPlus( ERRORED_SELECT_MSQ_QUERY, @@ -332,11 +372,11 @@ public static void init() throws Exception null, FAILURE_MSG ) - )); + ))); - Mockito.when(indexingServiceClient.getTaskStatus(ArgumentMatchers.eq(RUNNING_NON_MSQ_TASK))) - .thenReturn(new TaskStatusResponse( + Mockito.when(indexingServiceClient.taskStatus(ArgumentMatchers.eq(RUNNING_NON_MSQ_TASK))) + .thenReturn(Futures.immediateFuture(new TaskStatusResponse( RUNNING_NON_MSQ_TASK, new TaskStatusPlus( RUNNING_NON_MSQ_TASK, @@ -351,11 +391,11 @@ public static void init() throws Exception null, null ) - )); + ))); - Mockito.when(indexingServiceClient.getTaskStatus(ArgumentMatchers.eq(FAILED_NON_MSQ_TASK))) - .thenReturn(new TaskStatusResponse( + Mockito.when(indexingServiceClient.taskStatus(ArgumentMatchers.eq(FAILED_NON_MSQ_TASK))) + .thenReturn(Futures.immediateFuture(new TaskStatusResponse( FAILED_NON_MSQ_TASK, new TaskStatusPlus( FAILED_NON_MSQ_TASK, @@ -370,11 +410,11 @@ public static void init() throws Exception null, FAILURE_MSG ) - )); + ))); - Mockito.when(indexingServiceClient.getTaskStatus(ArgumentMatchers.eq(FINISHED_NON_MSQ_TASK))) - .thenReturn(new TaskStatusResponse( + Mockito.when(indexingServiceClient.taskStatus(ArgumentMatchers.eq(FINISHED_NON_MSQ_TASK))) + .thenReturn(Futures.immediateFuture(new TaskStatusResponse( FINISHED_NON_MSQ_TASK, new TaskStatusPlus( FINISHED_NON_MSQ_TASK, @@ -389,11 +429,11 @@ public static void init() throws Exception null, null ) - )); + ))); - Mockito.when(indexingServiceClient.getTaskStatus(ArgumentMatchers.eq(ACCEPTED_INSERT_MSQ_TASK))) - .thenReturn(new TaskStatusResponse( + Mockito.when(indexingServiceClient.taskStatus(ArgumentMatchers.eq(ACCEPTED_INSERT_MSQ_TASK))) + .thenReturn(Futures.immediateFuture(new TaskStatusResponse( ACCEPTED_SELECT_MSQ_QUERY, new TaskStatusPlus( ACCEPTED_SELECT_MSQ_QUERY, @@ -408,13 +448,16 @@ public static void init() throws Exception null, null ) - )); + ))); - Mockito.when(indexingServiceClient.getTaskPayload(ArgumentMatchers.eq(ACCEPTED_INSERT_MSQ_TASK))) - .thenReturn(new TaskPayloadResponse(ACCEPTED_INSERT_MSQ_TASK, MSQ_CONTROLLER_INSERT_PAYLOAD)); + Mockito.when(indexingServiceClient.taskPayload(ArgumentMatchers.eq(ACCEPTED_INSERT_MSQ_TASK))) + .thenReturn(Futures.immediateFuture(new TaskPayloadResponse( + ACCEPTED_INSERT_MSQ_TASK, + MSQ_CONTROLLER_INSERT_PAYLOAD + ))); - Mockito.when(indexingServiceClient.getTaskStatus(ArgumentMatchers.eq(RUNNING_INSERT_MSQ_QUERY))) - .thenReturn(new TaskStatusResponse( + Mockito.when(indexingServiceClient.taskStatus(ArgumentMatchers.eq(RUNNING_INSERT_MSQ_QUERY))) + .thenReturn(Futures.immediateFuture(new TaskStatusResponse( RUNNING_INSERT_MSQ_QUERY, new TaskStatusPlus( RUNNING_INSERT_MSQ_QUERY, @@ -429,14 +472,17 @@ public static void init() throws Exception null, null ) - )); + ))); - Mockito.when(indexingServiceClient.getTaskPayload(RUNNING_INSERT_MSQ_QUERY)) - .thenReturn(new TaskPayloadResponse(RUNNING_INSERT_MSQ_QUERY, MSQ_CONTROLLER_INSERT_PAYLOAD)); + Mockito.when(indexingServiceClient.taskPayload(RUNNING_INSERT_MSQ_QUERY)) + .thenReturn(Futures.immediateFuture(new TaskPayloadResponse( + RUNNING_INSERT_MSQ_QUERY, + MSQ_CONTROLLER_INSERT_PAYLOAD + ))); - Mockito.when(indexingServiceClient.getTaskStatus(ArgumentMatchers.eq(FINISHED_INSERT_MSQ_QUERY))) - .thenReturn(new TaskStatusResponse( + Mockito.when(indexingServiceClient.taskStatus(ArgumentMatchers.eq(FINISHED_INSERT_MSQ_QUERY))) + .thenReturn(Futures.immediateFuture(new TaskStatusResponse( FINISHED_INSERT_MSQ_QUERY, new TaskStatusPlus( FINISHED_INSERT_MSQ_QUERY, @@ -451,14 +497,24 @@ public static void init() throws Exception null, null ) - )); - - Mockito.when(indexingServiceClient.getTaskPayload(FINISHED_INSERT_MSQ_QUERY)) - .thenReturn(new TaskPayloadResponse(FINISHED_INSERT_MSQ_QUERY, MSQ_CONTROLLER_INSERT_PAYLOAD)); - + ))); + + Mockito.when(indexingServiceClient.taskReportAsMap(ArgumentMatchers.eq(FINISHED_INSERT_MSQ_QUERY))) + .thenReturn(Futures.immediateFuture(mapper.readValue( + mapper.writeValueAsString(TaskReport.buildTaskReports(MSQ_INSERT_TASK_REPORT)), + new TypeReference>() + { + } + ))); + + Mockito.when(indexingServiceClient.taskPayload(FINISHED_INSERT_MSQ_QUERY)) + .thenReturn(Futures.immediateFuture(new TaskPayloadResponse( + FINISHED_INSERT_MSQ_QUERY, + MSQ_CONTROLLER_INSERT_PAYLOAD + ))); - Mockito.when(indexingServiceClient.getTaskStatus(ArgumentMatchers.eq(ERRORED_INSERT_MSQ_QUERY))) - .thenReturn(new TaskStatusResponse( + Mockito.when(indexingServiceClient.taskStatus(ArgumentMatchers.eq(ERRORED_INSERT_MSQ_QUERY))) + .thenReturn(Futures.immediateFuture(new TaskStatusResponse( ERRORED_INSERT_MSQ_QUERY, new TaskStatusPlus( ERRORED_INSERT_MSQ_QUERY, @@ -473,18 +529,9 @@ public static void init() throws Exception null, FAILURE_MSG ) - )); - - resource = new SqlStatementResource( - new SqlStatementFactory(null), - new ServerConfig(), - CalciteTests.TEST_AUTHORIZER_MAPPER, - JSON_MAPPER, - indexingServiceClient - ); + ))); } - @Test public void testMSQSelectAcceptedQuery() { @@ -565,7 +612,6 @@ public void testFinishedSelectMSQQuery() throws Exception 100L, new ResultSetInformation( null, - false, 2L, null, RESULT_ROWS.stream().map(Arrays::asList).collect(Collectors.toList()) @@ -666,6 +712,77 @@ public void testFailedMSQQuery() } + @Test + public void testFinishedInsertMSQQuery() throws Exception + { + Response response = resource.doGetStatus(FINISHED_INSERT_MSQ_QUERY, makeOkRequest()); + Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); + Assert.assertEquals(new SqlStatementResult( + FINISHED_INSERT_MSQ_QUERY, + SqlStatementState.SUCCESS, + CREATED_TIME, + null, + 100L, + null, + null + ), response.getEntity()); + + Response resultsResponse = resource.doGetResults( + FINISHED_INSERT_MSQ_QUERY, + null, + null, + null, + null, + makeOkRequest() + ); + Assert.assertEquals( + Response.Status.OK.getStatusCode(), resultsResponse.getStatus()); + + + Assert.assertNull( + getResultRowsFromResponse(resource.doGetResults( + FINISHED_INSERT_MSQ_QUERY, + 1L, + null, + null, + null, + makeOkRequest() + )) + ); + Assert.assertNull( + getResultRowsFromResponse(resource.doGetResults( + FINISHED_INSERT_MSQ_QUERY, + 0L, + 1L, + null, + null, + makeOkRequest() + )) + ); + Assert.assertNull( + getResultRowsFromResponse(resource.doGetResults( + FINISHED_INSERT_MSQ_QUERY, + 0L, + 3L, + null, + null, + makeOkRequest() + )) + ); + + Assert.assertEquals( + Response.Status.PRECONDITION_FAILED.getStatusCode(), + resource.doGetResults(FINISHED_INSERT_MSQ_QUERY, -1L, 3L, null, null, makeOkRequest()) + .getStatus() + ); + Assert.assertEquals( + Response.Status.PRECONDITION_FAILED.getStatusCode(), + resource.doGetResults(FINISHED_INSERT_MSQ_QUERY, null, -1L, null, null, makeOkRequest()) + .getStatus() + ); + + } + @Test public void testNonMSQTasks() { @@ -738,13 +855,13 @@ public void testMSQInsertRunningQuery() } - private static void assertNullResponse(Response response, Response.Status expectectedStatus) + public static void assertNullResponse(Response response, Response.Status expectectedStatus) { Assert.assertEquals(expectectedStatus.getStatusCode(), response.getStatus()); Assert.assertNull(response.getEntity()); } - private static void assertExceptionMessage( + public static void assertExceptionMessage( Response response, String exceptionMessage, Response.Status expectectedStatus @@ -754,9 +871,13 @@ private static void assertExceptionMessage( Assert.assertEquals(exceptionMessage, getQueryExceptionFromResponse(response).getMessage()); } - private static List getResultRowsFromResponse(Response resultsResponse) throws IOException + public static List getResultRowsFromResponse(Response resultsResponse) throws IOException { - return JSON_MAPPER.readValue(SqlResourceTest.responseToByteArray(resultsResponse), List.class); + byte[] bytes = SqlResourceTest.responseToByteArray(resultsResponse); + if (bytes == null) { + return null; + } + return JSON_MAPPER.readValue(bytes, List.class); } private static QueryException getQueryExceptionFromResponse(Response response) @@ -768,17 +889,17 @@ private static QueryException getQueryExceptionFromResponse(Response response) } } - private MockHttpServletRequest makeOkRequest() + public static MockHttpServletRequest makeOkRequest() { return makeExpectedReq(CalciteTests.REGULAR_USER_AUTH_RESULT); } - private MockHttpServletRequest makeOtherUserRequest() + public static MockHttpServletRequest makeOtherUserRequest() { return makeExpectedReq(CalciteTests.SUPER_USER_AUTH_RESULT); } - private MockHttpServletRequest makeExpectedReq(AuthenticationResult authenticationResult) + public static MockHttpServletRequest makeExpectedReq(AuthenticationResult authenticationResult) { MockHttpServletRequest req = new MockHttpServletRequest(); req.attributes.put(AuthConfig.DRUID_AUTHENTICATION_RESULT, authenticationResult); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index e44d1974b8a3..5747d0d206df 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -227,6 +227,7 @@ public class MSQTestBase extends BaseCalciteQueryTest .put(QueryContexts.CTX_SQL_STRINGIFY_ARRAYS, false) .put(MultiStageQueryContext.CTX_MAX_NUM_TASKS, 2) .put(MSQWarnings.CTX_MAX_PARSE_EXCEPTIONS_ALLOWED, 0) + .put(MSQTaskQueryMaker.USER_KEY, "allowAll") .build(); public static final Map DURABLE_STORAGE_MSQ_CONTEXT = @@ -267,10 +268,10 @@ public class MSQTestBase extends BaseCalciteQueryTest protected File localFileStorageDir; protected LocalFileStorageConnector localFileStorageConnector; private static final Logger log = new Logger(MSQTestBase.class); - private ObjectMapper objectMapper; - private MSQTestOverlordServiceClient indexingServiceClient; + protected ObjectMapper objectMapper; + protected MSQTestOverlordServiceClient indexingServiceClient; protected MSQTestTaskActionClient testTaskActionClient; - private SqlStatementFactory sqlStatementFactory; + protected SqlStatementFactory sqlStatementFactory; private IndexIO indexIO; private MSQTestSegmentManager segmentManager; @@ -1080,7 +1081,7 @@ public void verifyResults() verifyWorkerCount(reportPayload.getCounters()); verifyCounters(reportPayload.getCounters()); - MSQSpec foundSpec = indexingServiceClient.getQuerySpecForTask(controllerId); + MSQSpec foundSpec = indexingServiceClient.getMSQControllerTask(controllerId).getQuerySpec(); log.info( "found generated segments: %s", segmentManager.getAllDataSegments().stream().map(s -> s.toString()).collect( @@ -1299,7 +1300,7 @@ public Pair, List>> log.info("found row signature %s", payload.getResults().getSignature()); log.info(rows.stream().map(Arrays::toString).collect(Collectors.joining("\n"))); - final MSQSpec spec = indexingServiceClient.getQuerySpecForTask(controllerId); + final MSQSpec spec = indexingServiceClient.getMSQControllerTask(controllerId).getQuerySpec(); log.info("Found spec: %s", objectMapper.writerWithDefaultPrettyPrinter().writeValueAsString(spec)); return new Pair<>(spec, Pair.of(payload.getResults().getSignature(), rows)); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestOverlordServiceClient.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestOverlordServiceClient.java index a4aff4d8756a..8494d6014718 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestOverlordServiceClient.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestOverlordServiceClient.java @@ -19,24 +19,30 @@ package org.apache.druid.msq.test; +import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableMap; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.SettableFuture; import com.google.inject.Injector; import org.apache.druid.client.indexing.NoopOverlordClient; +import org.apache.druid.client.indexing.TaskPayloadResponse; +import org.apache.druid.client.indexing.TaskStatusResponse; +import org.apache.druid.indexer.TaskLocation; +import org.apache.druid.indexer.TaskState; +import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.indexing.common.TaskReport; import org.apache.druid.indexing.common.actions.TaskActionClient; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.msq.exec.Controller; import org.apache.druid.msq.exec.ControllerImpl; import org.apache.druid.msq.exec.WorkerMemoryParameters; import org.apache.druid.msq.indexing.MSQControllerTask; -import org.apache.druid.msq.indexing.MSQSpec; -import org.apache.druid.msq.indexing.report.MSQTaskReport; +import org.joda.time.DateTime; import javax.annotation.Nullable; +import java.io.IOException; import java.util.HashMap; import java.util.Map; @@ -48,7 +54,12 @@ public class MSQTestOverlordServiceClient extends NoopOverlordClient private final WorkerMemoryParameters workerMemoryParameters; private Map inMemoryControllers = new HashMap<>(); private Map> reports = new HashMap<>(); - private Map msqSpec = new HashMap<>(); + private Map inMemoryControllerTask = new HashMap<>(); + + public static final DateTime CREATED_TIME = DateTimes.of("2023-05-31T12:00Z"); + public static final DateTime QUEUE_INSERTION_TIME = DateTimes.of("2023-05-31T12:01Z"); + + public static final long DURATION = 100L; public MSQTestOverlordServiceClient( ObjectMapper objectMapper, @@ -77,14 +88,11 @@ public ListenableFuture runTask(String taskId, Object taskObject) ); MSQControllerTask cTask = objectMapper.convertValue(taskObject, MSQControllerTask.class); - msqSpec.put(cTask.getId(), cTask.getQuerySpec()); + inMemoryControllerTask.put(cTask.getId(), cTask); - controller = new ControllerImpl( - cTask, - msqTestControllerContext - ); + controller = new ControllerImpl(cTask, msqTestControllerContext); - inMemoryControllers.put(cTask.getId(), controller); + inMemoryControllers.put(controller.id(), controller); controller.run(); return Futures.immediateFuture(null); @@ -110,24 +118,60 @@ public ListenableFuture cancelTask(String taskId) public ListenableFuture> taskReportAsMap(String taskId) { SettableFuture> future = SettableFuture.create(); - future.set( - ImmutableMap.of( - MSQTaskReport.REPORT_KEY, - getReportForTask(taskId).get(MSQTaskReport.REPORT_KEY) - )); + try { + future.set( + objectMapper.readValue( + objectMapper.writeValueAsBytes(getReportForTask(taskId)), + new TypeReference>() + { + } + )); + } + catch (IOException e) { + throw new RuntimeException(e); + } + return future; + } + + @Override + public ListenableFuture taskPayload(String taskId) + { + SettableFuture future = SettableFuture.create(); + future.set(new TaskPayloadResponse(taskId, getMSQControllerTask(taskId))); + return future; + } + + @Override + public ListenableFuture taskStatus(String taskId) + { + SettableFuture future = SettableFuture.create(); + future.set(new TaskStatusResponse(taskId, new TaskStatusPlus( + taskId, + null, + MSQControllerTask.TYPE, + CREATED_TIME, + QUEUE_INSERTION_TIME, + TaskState.SUCCESS, + null, + DURATION, + new TaskLocation("localhost", 1, 1), + null, + null + ))); + return future; } // hooks to pull stuff out for testing @Nullable - Map getReportForTask(String id) + public Map getReportForTask(String id) { return reports.get(id); } @Nullable - MSQSpec getQuerySpecForTask(String id) + MSQControllerTask getMSQControllerTask(String id) { - return msqSpec.get(id); + return inMemoryControllerTask.get(id); } } diff --git a/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClient.java b/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClient.java index 85344e8cdbd9..dac5dfc2263c 100644 --- a/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClient.java +++ b/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClient.java @@ -20,6 +20,7 @@ package org.apache.druid.rpc.indexing; import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.client.indexing.TaskPayloadResponse; import org.apache.druid.client.indexing.TaskStatusResponse; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.rpc.ServiceRetryPolicy; @@ -52,5 +53,7 @@ public interface OverlordClient ListenableFuture> taskReportAsMap(String taskId); + ListenableFuture taskPayload(String taskId); + OverlordClient withRetryPolicy(ServiceRetryPolicy retryPolicy); } diff --git a/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClientImpl.java b/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClientImpl.java index 4b9cfd7433fd..5c28d87a8d11 100644 --- a/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClientImpl.java +++ b/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClientImpl.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.client.indexing.TaskPayloadResponse; import org.apache.druid.client.indexing.TaskStatusResponse; import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.indexer.TaskStatus; @@ -131,6 +132,22 @@ public ListenableFuture> taskReportAsMap(String taskId) ); } + @Override + public ListenableFuture taskPayload(String taskId) + { + final String path = StringUtils.format("/druid/indexer/v1/task/%s", StringUtils.urlEncode(taskId)); + + return FutureUtils.transform( + client.asyncRequest( + new RequestBuilder(HttpMethod.GET, path), + new BytesFullResponseHandler() + ), + holder -> deserialize(holder, new TypeReference() + { + }) + ); + } + @Override public OverlordClientImpl withRetryPolicy(ServiceRetryPolicy retryPolicy) { diff --git a/server/src/test/java/org/apache/druid/client/indexing/NoopOverlordClient.java b/server/src/test/java/org/apache/druid/client/indexing/NoopOverlordClient.java index 907973edcf72..dbfa1ec10473 100644 --- a/server/src/test/java/org/apache/druid/client/indexing/NoopOverlordClient.java +++ b/server/src/test/java/org/apache/druid/client/indexing/NoopOverlordClient.java @@ -59,6 +59,12 @@ public ListenableFuture> taskReportAsMap(String taskId) throw new UnsupportedOperationException(); } + @Override + public ListenableFuture taskPayload(String taskId) + { + throw new UnsupportedOperationException(); + } + @Override public OverlordClient withRetryPolicy(ServiceRetryPolicy retryPolicy) { diff --git a/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java b/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java index 7a899dd5a913..c6011c39bef1 100644 --- a/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java +++ b/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java @@ -2038,6 +2038,10 @@ private T deserializeResponse(Response resp, Class clazz) throws IOExcept public static byte[] responseToByteArray(Response resp) throws IOException { ByteArrayOutputStream baos = new ByteArrayOutputStream(); + + if (resp.getEntity() == null) { + return null; + } ((StreamingOutput) resp.getEntity()).write(baos); return baos.toByteArray(); } From e37ca026533f5ff47a54110641af3e2e7aff8cda Mon Sep 17 00:00:00 2001 From: cryptoe Date: Wed, 21 Jun 2023 17:39:24 +0530 Subject: [PATCH 04/15] 1. Adding native types to response. 2. Adding rows and size to response. 3. Adding dataSource to response. 4. Adding exceptionDetails to response. --- .../druid/msq/indexing/MSQControllerTask.java | 15 + .../error/InsertCannotBeEmptyFault.java | 2 +- .../druid/msq/sql/MSQTaskQueryMaker.java | 4 +- .../druid/msq/sql/entity/ColNameAndType.java | 22 +- .../msq/sql/entity/ResultSetInformation.java | 47 ++- .../sql/resources/SqlStatementResource.java | 385 +++++++++++------- .../msq/sql/resources/SqlTaskResource.java | 10 +- .../msq/indexing/MSQControllerTaskTest.java | 1 + .../sql/SqlMsqStatementResourcePostTest.java | 60 ++- .../msq/sql/SqlStatementResourceTest.java | 28 +- .../druid/msq/sql/SqlTaskStatusTest.java | 3 +- .../test/MSQTestOverlordServiceClient.java | 13 +- .../apache/druid/query/BadQueryException.java | 2 +- .../query/QueryCapacityExceededException.java | 6 +- .../apache/druid/query/QueryException.java | 59 ++- .../query/QueryInterruptedException.java | 2 +- .../druid/query/QueryTimeoutException.java | 8 +- .../query/QueryUnsupportedException.java | 4 +- .../druid/query/QueryExceptionTest.java | 6 +- .../apache/druid/server/QueryResource.java | 3 +- .../druid/client/JsonParserIteratorTest.java | 13 +- .../druid/sql/calcite/rel/Windowing.java | 1 + .../druid/sql/avatica/ErrorHandlerTest.java | 2 +- 23 files changed, 469 insertions(+), 227 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java index fb1bc13384cc..022805257381 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java @@ -50,6 +50,7 @@ import org.apache.druid.rpc.ServiceClientFactory; import org.apache.druid.rpc.StandardRetryPolicy; import org.apache.druid.rpc.indexing.OverlordClient; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.server.security.ResourceAction; import org.apache.druid.sql.calcite.run.SqlResults; import org.joda.time.Interval; @@ -92,6 +93,9 @@ public class MSQControllerTask extends AbstractTask implements ClientTaskQuery @Nullable private final List sqlTypeNames; + @Nullable + private final List nativeTypeNames; + // Using an Injector directly because tasks do not have a way to provide their own Guice modules. @JacksonInject private Injector injector; @@ -106,6 +110,7 @@ public MSQControllerTask( @JsonProperty("sqlQueryContext") @Nullable Map sqlQueryContext, @JsonProperty("sqlResultsContext") @Nullable SqlResults.Context sqlResultsContext, @JsonProperty("sqlTypeNames") @Nullable List sqlTypeNames, + @JsonProperty("nativeTypeNames") @Nullable List nativeTypeNames, @JsonProperty("context") @Nullable Map context ) { @@ -122,6 +127,7 @@ public MSQControllerTask( this.sqlQueryContext = sqlQueryContext; this.sqlResultsContext = sqlResultsContext; this.sqlTypeNames = sqlTypeNames; + this.nativeTypeNames = nativeTypeNames; addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, true); } @@ -155,6 +161,15 @@ public List getSqlTypeNames() return sqlTypeNames; } + + @Nullable + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public List getNativeTypeNames() + { + return nativeTypeNames; + } + @Nullable @JsonProperty @JsonInclude(JsonInclude.Include.NON_NULL) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertCannotBeEmptyFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertCannotBeEmptyFault.java index 4285ace4180b..7948e8136150 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertCannotBeEmptyFault.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InsertCannotBeEmptyFault.java @@ -29,7 +29,7 @@ @JsonTypeName(InsertCannotBeEmptyFault.CODE) public class InsertCannotBeEmptyFault extends BaseMSQFault { - static final String CODE = "InsertCannotBeEmpty"; + public static final String CODE = "InsertCannotBeEmpty"; private final String dataSource; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java index 0bfb4341b92e..c0de08a809f8 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java @@ -179,6 +179,7 @@ public QueryResponse runQuery(final DruidQuery druidQuery) finalizeAggregations ? null /* Not needed */ : buildAggregationIntermediateTypeMap(druidQuery); final List sqlTypeNames = new ArrayList<>(); + final List columnTypeList = new ArrayList<>(); final List columnMappings = QueryUtils.buildColumnMappings(fieldMapping, druidQuery); for (final Pair entry : fieldMapping) { @@ -192,8 +193,8 @@ public QueryResponse runQuery(final DruidQuery druidQuery) } else { sqlTypeName = druidQuery.getOutputRowType().getFieldList().get(entry.getKey()).getType().getSqlTypeName(); } - sqlTypeNames.add(sqlTypeName); + columnTypeList.add(druidQuery.getOutputRowSignature().getColumnType(queryColumn).orElse(ColumnType.STRING)); } final MSQDestination destination; @@ -253,6 +254,7 @@ public QueryResponse runQuery(final DruidQuery druidQuery) plannerContext.queryContextMap(), SqlResults.Context.fromPlannerContext(plannerContext), sqlTypeNames, + columnTypeList, null ); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ColNameAndType.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ColNameAndType.java index 29c1a686a2c0..73b9d246aa8a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ColNameAndType.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ColNameAndType.java @@ -31,12 +31,19 @@ public class ColNameAndType private final String colName; private final String sqlTypeName; + private final String nativeTypeName; + @JsonCreator - public ColNameAndType(@JsonProperty("name") String colName, @JsonProperty("type") String sqlTypeName) + public ColNameAndType( + @JsonProperty("name") String colName, + @JsonProperty("type") String sqlTypeName, + @JsonProperty("nativeType") String nativeTypeName + ) { this.colName = colName; this.sqlTypeName = sqlTypeName; + this.nativeTypeName = nativeTypeName; } @JsonProperty("name") @@ -51,6 +58,12 @@ public String getSqlTypeName() return sqlTypeName; } + @JsonProperty("nativeType") + public String getNativeTypeName() + { + return nativeTypeName; + } + @Override public boolean equals(Object o) { @@ -61,13 +74,15 @@ public boolean equals(Object o) return false; } ColNameAndType that = (ColNameAndType) o; - return Objects.equals(colName, that.colName) && Objects.equals(sqlTypeName, that.sqlTypeName); + return Objects.equals(colName, that.colName) + && Objects.equals(sqlTypeName, that.sqlTypeName) + && Objects.equals(nativeTypeName, that.nativeTypeName); } @Override public int hashCode() { - return Objects.hash(colName, sqlTypeName); + return Objects.hash(colName, sqlTypeName, nativeTypeName); } @Override @@ -76,6 +91,7 @@ public String toString() return "ColNameAndType{" + "colName='" + colName + '\'' + ", sqlTypeName='" + sqlTypeName + '\'' + + ", nativeTypeName='" + nativeTypeName + '\'' + '}'; } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ResultSetInformation.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ResultSetInformation.java index 2eceb6b8796d..0ecfc17c0853 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ResultSetInformation.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ResultSetInformation.java @@ -32,9 +32,9 @@ public class ResultSetInformation { @Nullable - private final Long totalRows; + private final Long numRows; @Nullable - private final Long totalSize; + private final Long sizeInBytes; private final ResultFormat resultFormat; @@ -42,36 +42,41 @@ public class ResultSetInformation @Nullable private final List records; + @Nullable + private final String dataSource; + @JsonCreator public ResultSetInformation( @Nullable @JsonProperty ResultFormat resultFormat, - @JsonProperty @Nullable Long totalRows, - @JsonProperty @Nullable Long totalSize, + @JsonProperty @Nullable Long numRows, + @JsonProperty @Nullable Long sizeInBytes, + @JsonProperty @Nullable String dataSource, @JsonProperty("sampleRecords") @Nullable List records ) { - this.totalRows = totalRows; - this.totalSize = totalSize; + this.numRows = numRows; + this.sizeInBytes = sizeInBytes; this.resultFormat = resultFormat; + this.dataSource = dataSource; this.records = records; } @Nullable @JsonProperty @JsonInclude(JsonInclude.Include.NON_NULL) - public Long getTotalRows() + public Long getNumRows() { - return totalRows; + return numRows; } @Nullable @JsonProperty @JsonInclude(JsonInclude.Include.NON_NULL) - public Long getTotalSize() + public Long getSizeInBytes() { - return totalSize; + return sizeInBytes; } @JsonProperty @@ -82,6 +87,13 @@ public ResultFormat getResultFormat() return resultFormat; } + @JsonProperty + @Nullable + @JsonInclude(JsonInclude.Include.NON_NULL) + public String getDataSource() + { + return dataSource; + } @Nullable @JsonProperty("sampleRecords") @@ -102,27 +114,30 @@ public boolean equals(Object o) return false; } ResultSetInformation that = (ResultSetInformation) o; - return Objects.equals(totalRows, that.totalRows) - && Objects.equals(totalSize, that.totalSize) + return Objects.equals(numRows, that.numRows) + && Objects.equals(sizeInBytes, that.sizeInBytes) && resultFormat == that.resultFormat - && Objects.equals(records, that.records); + && Objects.equals(records, that.records) + && Objects.equals(dataSource, that.dataSource); } @Override public int hashCode() { - return Objects.hash(totalRows, totalSize, resultFormat, records); + return Objects.hash(numRows, sizeInBytes, resultFormat, records, dataSource); } @Override public String toString() { return "ResultSetInformation{" + - "totalRows=" + totalRows + - ", totalSize=" + totalSize + + "totalRows=" + numRows + + ", totalSize=" + sizeInBytes + ", resultFormat=" + resultFormat + ", records=" + records + + ", dataSource='" + dataSource + '\'' + '}'; } + } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java index eb3f70fd20b3..959492f18500 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java @@ -36,6 +36,7 @@ import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.guava.Sequence; @@ -60,6 +61,7 @@ import org.apache.druid.query.QueryUnsupportedException; import org.apache.druid.query.ResourceLimitExceededException; import org.apache.druid.rpc.indexing.OverlordClient; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.server.QueryResponse; import org.apache.druid.server.initialization.ServerConfig; import org.apache.druid.server.security.Access; @@ -94,6 +96,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; @@ -151,8 +154,7 @@ public Response doGetEnabled(@Context final HttpServletRequest request) @Produces(MediaType.APPLICATION_JSON) @Consumes(MediaType.APPLICATION_JSON) public Response doPost( - final SqlQuery sqlQuery, - @Context final HttpServletRequest req + final SqlQuery sqlQuery, @Context final HttpServletRequest req ) { final HttpStatement stmt = msqSqlStatementFactory.httpStatement(sqlQuery, req); @@ -176,6 +178,7 @@ public Response doPost( ExecutionMode.ASYNC ), null, + null, null ), stmt.sqlQueryId() @@ -252,8 +255,7 @@ public Response doPost( @Path("/{id}") @Produces(MediaType.APPLICATION_JSON) public Response doGetStatus( - @PathParam("id") final String queryId, - @Context final HttpServletRequest req + @PathParam("id") final String queryId, @Context final HttpServletRequest req ) { try { @@ -272,8 +274,7 @@ public Response doGetStatus( } } catch (ForbiddenException e) { - throw (ForbiddenException) serverConfig.getErrorResponseTransformStrategy() - .transformIfNeeded(e); + throw (ForbiddenException) serverConfig.getErrorResponseTransformStrategy().transformIfNeeded(e); } catch (QueryException e) { return buildNonOkResponse(Response.Status.INTERNAL_SERVER_ERROR.getStatusCode(), e, queryId); @@ -299,14 +300,26 @@ public Response doGetResults( if (offset != null && offset < 0) { return buildNonOkResponse( Response.Status.PRECONDITION_FAILED.getStatusCode(), - new QueryException(null, "offset cannot be negative. Please pass a positive number.", null, null), + new QueryException( + null, + "offset cannot be negative. Please pass a positive number.", + null, + null, + null + ), queryId ); } if (numberOfRows != null && numberOfRows < 0) { return buildNonOkResponse( Response.Status.PRECONDITION_FAILED.getStatusCode(), - new QueryException(null, "numRows cannot be negative. Please pass a positive number.", null, null), + new QueryException( + null, + "numRows cannot be negative. Please pass a positive number.", + null, + null, + null + ), queryId ); } @@ -330,20 +343,20 @@ public Response doGetResults( SqlStatementState sqlStatementState = getSqlStatementState(statusPlus); if (sqlStatementState == SqlStatementState.RUNNING || sqlStatementState == SqlStatementState.ACCEPTED) { - return buildNonOkResponse( - Response.Status.NOT_FOUND.getStatusCode(), - new QueryException( - null, - StringUtils.format("Query is [%s]. Please wait for it to complete.", sqlStatementState), - null, - null + return buildNonOkResponse(Response.Status.NOT_FOUND.getStatusCode(), new QueryException( + null, + StringUtils.format( + "Query is [%s]. Please wait for it to complete.", + sqlStatementState ), - queryId - ); + null, + null, + null + ), queryId); } else if (sqlStatementState == SqlStatementState.FAILED) { return buildNonOkResponse( Response.Status.NOT_FOUND.getStatusCode(), - new QueryException(null, statusPlus.getErrorMsg(), null, null), + new QueryException(null, statusPlus.getErrorMsg(), null, null, null), queryId ); } else { @@ -352,38 +365,35 @@ public Response doGetResults( if (!signature.isPresent()) { return Response.ok().build(); } - Optional> results = getResults(overlordWork(overlordClient.taskReportAsMap(queryId))); - - return Response.ok( - (StreamingOutput) outputStream -> { - CountingOutputStream os = new CountingOutputStream(outputStream); + Optional> results = getResults(getPayload(overlordWork(overlordClient.taskReportAsMap(queryId)))); - try (final ResultFormat.Writer writer = ResultFormat.OBJECT.createFormatter(os, jsonMapper)) { - List rowSignature = signature.get(); - writer.writeResponseStart(); + return Response.ok((StreamingOutput) outputStream -> { + CountingOutputStream os = new CountingOutputStream(outputStream); - for (int k = (int) start; k < Math.min(last, results.get().size()); k++) { - writer.writeRowStart(); - for (int i = 0; i < rowSignature.size(); i++) { - writer.writeRowField(rowSignature.get(i).getColName(), ((List) results.get().get(k)).get(i)); - } - writer.writeRowEnd(); - } + try (final ResultFormat.Writer writer = ResultFormat.OBJECT.createFormatter(os, jsonMapper)) { + List rowSignature = signature.get(); + writer.writeResponseStart(); - writer.writeResponseEnd(); - } - catch (Exception e) { - log.error(e, "Unable to stream results back for query[%s]", queryId); - throw new ISE(e, "Unable to stream results back for query[%s]", queryId); + for (int k = (int) start; k < Math.min(last, results.get().size()); k++) { + writer.writeRowStart(); + for (int i = 0; i < rowSignature.size(); i++) { + writer.writeRowField(rowSignature.get(i).getColName(), ((List) results.get().get(k)).get(i)); } + writer.writeRowEnd(); } - ).build(); + + writer.writeResponseEnd(); + } + catch (Exception e) { + log.error(e, "Unable to stream results back for query[%s]", queryId); + throw new ISE(e, "Unable to stream results back for query[%s]", queryId); + } + }).build(); } } catch (ForbiddenException e) { - throw (ForbiddenException) serverConfig.getErrorResponseTransformStrategy() - .transformIfNeeded(e); + throw (ForbiddenException) serverConfig.getErrorResponseTransformStrategy().transformIfNeeded(e); } catch (QueryException e) { return buildNonOkResponse(Response.Status.INTERNAL_SERVER_ERROR.getStatusCode(), e, queryId); @@ -438,8 +448,7 @@ public Response deleteQuery(@PathParam("id") final String queryId, @Context fina } catch (ForbiddenException e) { - throw (ForbiddenException) serverConfig.getErrorResponseTransformStrategy() - .transformIfNeeded(e); + throw (ForbiddenException) serverConfig.getErrorResponseTransformStrategy().transformIfNeeded(e); } catch (QueryException e) { return buildNonOkResponse(Response.Status.INTERNAL_SERVER_ERROR.getStatusCode(), e, queryId); @@ -451,55 +460,48 @@ public Response deleteQuery(@PathParam("id") final String queryId, @Context fina } private Response buildStandardResponse( - Sequence sequence, - SqlQuery sqlQuery, - String sqlQueryId, - SqlRowTransformer rowTransformer + Sequence sequence, SqlQuery sqlQuery, String sqlQueryId, SqlRowTransformer rowTransformer ) throws IOException { final Yielder yielder0 = Yielders.each(sequence); try { - final Response.ResponseBuilder responseBuilder = Response - .ok( - (StreamingOutput) outputStream -> { - CountingOutputStream os = new CountingOutputStream(outputStream); - Yielder yielder = yielder0; - - try (final ResultFormat.Writer writer = sqlQuery.getResultFormat() - .createFormatter(os, jsonMapper)) { - writer.writeResponseStart(); - - if (sqlQuery.includeHeader()) { - writer.writeHeader( - rowTransformer.getRowType(), - sqlQuery.includeTypesHeader(), - sqlQuery.includeSqlTypesHeader() - ); - } - - while (!yielder.isDone()) { - final Object[] row = yielder.get(); - writer.writeRowStart(); - for (int i = 0; i < rowTransformer.getFieldList().size(); i++) { - final Object value = rowTransformer.transform(row, i); - writer.writeRowField(rowTransformer.getFieldList().get(i), value); - } - writer.writeRowEnd(); - yielder = yielder.next(null); - } - - writer.writeResponseEnd(); - } - catch (Exception e) { - log.error(e, "Unable to send SQL response [%s]", sqlQueryId); - throw new RuntimeException(e); - } - finally { - yielder.close(); - } - } - ); + final Response.ResponseBuilder responseBuilder = Response.ok((StreamingOutput) outputStream -> { + CountingOutputStream os = new CountingOutputStream(outputStream); + Yielder yielder = yielder0; + + try (final ResultFormat.Writer writer = sqlQuery.getResultFormat().createFormatter(os, jsonMapper)) { + writer.writeResponseStart(); + + if (sqlQuery.includeHeader()) { + writer.writeHeader( + rowTransformer.getRowType(), + sqlQuery.includeTypesHeader(), + sqlQuery.includeSqlTypesHeader() + ); + } + + while (!yielder.isDone()) { + final Object[] row = yielder.get(); + writer.writeRowStart(); + for (int i = 0; i < rowTransformer.getFieldList().size(); i++) { + final Object value = rowTransformer.transform(row, i); + writer.writeRowField(rowTransformer.getFieldList().get(i), value); + } + writer.writeRowEnd(); + yielder = yielder.next(null); + } + + writer.writeResponseEnd(); + } + catch (Exception e) { + log.error(e, "Unable to send SQL response [%s]", sqlQueryId); + throw new RuntimeException(e); + } + finally { + yielder.close(); + } + }); if (sqlQuery.includeHeader()) { responseBuilder.header(SqlResource.SQL_HEADER_RESPONSE_HEADER, SqlResource.SQL_HEADER_VALUE); @@ -537,6 +539,7 @@ private Response buildTaskResponse(Sequence sequence, String user) taskId ), null, + null, null ), taskId @@ -552,13 +555,8 @@ private Response buildNonOkResponse(int status, SanitizableException e, String s { // Though transformIfNeeded returns an exception, its purpose is to return // a QueryException - Exception cleaned = serverConfig - .getErrorResponseTransformStrategy() - .transformIfNeeded(e); - return Response - .status(status) - .entity(cleaned) - .build(); + Exception cleaned = serverConfig.getErrorResponseTransformStrategy().transformIfNeeded(e); + return Response.status(status).entity(cleaned).build(); } @@ -570,13 +568,21 @@ private static Optional> getSignature( if (msqControllerTask.getQuerySpec().getDestination().getClass() == TaskReportMSQDestination.class) { ColumnMappings columnMappings = msqControllerTask.getQuerySpec().getColumnMappings(); List sqlTypeNames = msqControllerTask.getSqlTypeNames(); - if (sqlTypeNames == null) { + if (sqlTypeNames == null || sqlTypeNames.size() != columnMappings.size()) { + return Optional.empty(); + } + List nativeTypeNames = msqControllerTask.getNativeTypeNames(); + if (nativeTypeNames == null || nativeTypeNames.size() != columnMappings.size()) { return Optional.empty(); } List signature = new ArrayList<>(columnMappings.size()); int index = 0; for (String colName : columnMappings.getOutputColumnNames()) { - signature.add(new ColNameAndType(colName, sqlTypeNames.get(index).getName())); + signature.add(new ColNameAndType( + colName, + sqlTypeNames.get(index).getName(), + nativeTypeNames.get(index).asTypeString() + )); index++; } return Optional.of(signature); @@ -588,48 +594,93 @@ private static Optional> getSignature( private void checkTaskPayloadOrThrow(TaskPayloadResponse taskPayloadResponse, String queryId) throws QueryException { if (taskPayloadResponse == null || taskPayloadResponse.getPayload() == null) { - throw new QueryException(QueryException.UNKNOWN_EXCEPTION_ERROR_CODE, - StringUtils.format( - "Could not get payload details of query[%s] from the overlord", - queryId - ), null, null + throw new QueryException( + QueryException.UNKNOWN_EXCEPTION_ERROR_CODE, + StringUtils.format( + "Could not get payload details for query[%s] from the overlord", + queryId + ), + null, + null, + null ); } if (MSQControllerTask.class != taskPayloadResponse.getPayload().getClass()) { - throw new QueryException(QueryException.UNKNOWN_EXCEPTION_ERROR_CODE, - StringUtils.format( - "Fetched an unexpected payload details of query[%s] from the overlord.", - queryId - ), null, null + throw new QueryException( + QueryException.UNKNOWN_EXCEPTION_ERROR_CODE, + StringUtils.format( + "Fetched an unexpected payload for query[%s] from the overlord.", + queryId + ), + null, + null, + null ); } } private Optional getSampleResults( - String asyncResultId, - boolean isSelectQuery, - SqlStatementState sqlStatementState + String asyncResultId, boolean isSelectQuery, String dataSource, SqlStatementState sqlStatementState ) { - // only populate sample results in case a select query is successful - if (isSelectQuery && sqlStatementState == SqlStatementState.SUCCESS) { - Map report = overlordWork(overlordClient.taskReportAsMap(asyncResultId)); - Optional> rows = getResults(report); - if (rows.isPresent()) { - return Optional.of(new ResultSetInformation( - null, - (long) rows.get().size(), - null, - rows.get() - )); - } else { + if (sqlStatementState == SqlStatementState.SUCCESS) { + Map payload = getPayload(overlordWork(overlordClient.taskReportAsMap(asyncResultId))); + Optional> rowsAndSize = getRowsAndSizeFromPayload(payload, isSelectQuery); + return Optional.of(new ResultSetInformation( + null, + // since the rows can be sampled, get the number of rows from counters + rowsAndSize.orElse(new Pair<>(null, null)).lhs, + rowsAndSize.orElse(new Pair<>(null, null)).rhs, + dataSource, + // only populate sample results in case a select query is successful + isSelectQuery ? getResults(payload).orElse(null) : null + )); + } else { + return Optional.empty(); + } + } + + private Optional> getRowsAndSizeFromPayload(Map payload, boolean isSelectQuery) + { + List stages = getList(payload, "stages"); + if (stages == null || stages.isEmpty()) { + return Optional.empty(); + } else { + int maxStage = stages.size() - 1; // Last stage output is the total number of rows returned to the end user. + Map counterMap = getMap(getMap(payload, "counters"), String.valueOf(maxStage)); + Long rows = -1L; + Long sizeInBytes = -1L; + if (counterMap == null) { return Optional.empty(); } + for (Map.Entry worker : counterMap.entrySet()) { + Object workerChannels = worker.getValue(); + if (workerChannels == null || !(workerChannels instanceof Map)) { + return Optional.empty(); + } + if (isSelectQuery) { + Object output = ((Map) workerChannels).get("output"); + if (output != null && output instanceof Map) { + List rowsPerChannel = (List) ((Map) output).get("rows"); + List bytesPerChannel = (List) ((Map) output).get("bytes"); + for (Integer row : rowsPerChannel) { + rows = rows + row; + } + for (Integer bytes : bytesPerChannel) { + sizeInBytes = sizeInBytes + bytes; + } + } + } else { + Object output = ((Map) workerChannels).get("segmentGenerationProgress"); + if (output != null && output instanceof Map) { + rows += (Integer) ((Map) output).get("rowsPushed"); + } + } + } - } else { - return Optional.empty(); + return Optional.of(new Pair<>(rows == -1L ? null : rows + 1, sizeInBytes == -1L ? null : sizeInBytes + 1)); } } @@ -650,35 +701,61 @@ private Optional getStatementStatus(String queryId, String c SqlStatementState sqlStatementState = getSqlStatementState(statusPlus); if (SqlStatementState.FAILED == sqlStatementState) { + Map exceptionDetails = getQueryExceptionDetails(getPayload(overlordWork(overlordClient.taskReportAsMap( + queryId)))); + Map exception = getMap(exceptionDetails, "error"); + if (exceptionDetails == null || exception == null) { + return Optional.of(new SqlStatementResult( + queryId, + sqlStatementState, + taskResponse.getStatus().getCreatedTime(), + null, + taskResponse.getStatus().getDuration(), + null, + new QueryException(null, statusPlus.getErrorMsg(), null, null, null) + )); + } + + final String errorMessage = String.valueOf(exception.getOrDefault("errorMessage", statusPlus.getErrorMsg())); + exception.remove("errorMessage"); + String host = (String) exceptionDetails.getOrDefault("host", null); + Map stringException = new HashMap<>(); + for (Map.Entry exceptionKeys : exception.entrySet()) { + stringException.put(exceptionKeys.getKey(), String.valueOf(exceptionKeys.getValue())); + } return Optional.of(new SqlStatementResult( queryId, sqlStatementState, - taskResponse.getStatus() - .getCreatedTime(), + taskResponse.getStatus().getCreatedTime(), null, taskResponse.getStatus().getDuration(), null, - new QueryException(null, statusPlus.getErrorMsg(), null, null) + new QueryException(null, errorMessage, null, host, stringException) )); - } + } else { + MSQControllerTask msqControllerTask = getMSQControllerTaskOrThrow(queryId, currentUser); + Optional> signature = getSignature(msqControllerTask); - MSQControllerTask msqControllerTask = getMSQControllerTaskOrThrow(queryId, currentUser); - Optional> signature = getSignature(msqControllerTask); + return Optional.of(new SqlStatementResult( + queryId, + sqlStatementState, + taskResponse.getStatus().getCreatedTime(), + signature.orElse(null), + taskResponse.getStatus().getDuration(), + withResults ? getSampleResults( + queryId, + signature.isPresent(), + msqControllerTask.getDataSource(), + sqlStatementState + ).orElse(null) : null, + null + )); + } + } - return Optional.of(new SqlStatementResult( - queryId, - sqlStatementState, - taskResponse.getStatus() - .getCreatedTime(), - signature.orElse(null), - taskResponse.getStatus().getDuration(), - withResults ? getSampleResults( - queryId, - signature.isPresent(), - sqlStatementState - ).orElse(null) : null, - null - )); + private Map getQueryExceptionDetails(Map payload) + { + return getMap(getMap(payload, "status"), "errorReport"); } private MSQControllerTask getMSQControllerTaskOrThrow(String queryId, String currentUser) throws ForbiddenException @@ -702,7 +779,7 @@ private T overlordWork(final ListenableFuture future) return FutureUtils.getUnchecked(future, true); } catch (RuntimeException e) { - throw new QueryException(null, "Unable to contact overlord " + e.getMessage(), null, null); + throw new QueryException(null, "Unable to contact overlord " + e.getMessage(), null, null, null); } } @@ -738,14 +815,21 @@ private Map getMap(Map map, String key) return (Map) map.get(key); } + @SuppressWarnings("rawtypes") + private List getList(Map map, String key) + { + if (map == null) { + return null; + } + return (List) map.get(key); + } + /** * Get results from report */ @SuppressWarnings("unchecked") - private Optional> getResults(Map results) + private Optional> getResults(Map payload) { - Map msqReport = getMap(results, "multiStageQuery"); - Map payload = getMap(msqReport, "payload"); Map resultsHolder = getMap(payload, "results"); if (resultsHolder == null) { @@ -760,6 +844,13 @@ private Optional> getResults(Map results) return Optional.of(rows); } + private Map getPayload(Map results) + { + Map msqReport = getMap(results, "multiStageQuery"); + Map payload = getMap(msqReport, "payload"); + return payload; + } + private static long getLastIndex(Long numberOfRows, long start) { final long last; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlTaskResource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlTaskResource.java index 02130f3e2dbd..7db6955a2108 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlTaskResource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlTaskResource.java @@ -71,15 +71,15 @@ /** * Endpoint for SQL execution using MSQ tasks. - * + *

* Unlike the SQL endpoint in {@link SqlResource}, this endpoint returns task IDs instead of inline results. Queries * are executed asynchronously using MSQ tasks via the indexing service (Overlord + MM or Indexer). This endpoint * does not provide a way for users to get the status or results of a query. That must be done using Overlord APIs * for status and reports. - * + *

* One exception: EXPLAIN query results are returned inline by this endpoint, in the same way as {@link SqlResource} * would return them. - * + *

* This endpoint does not support system tables or INFORMATION_SCHEMA. Queries on those tables result in errors. */ @Path("/druid/v2/sql/task/") @@ -131,7 +131,7 @@ public Response doGetEnabled(@Context final HttpServletRequest request) /** * Post a query task. - * + *

* Execution uses {@link MSQTaskSqlEngine} to ship the query off to the Overlord as an indexing task using * {@link org.apache.druid.msq.indexing.MSQControllerTask}. The task ID is returned immediately to the caller, * and execution proceeds asynchronously. @@ -333,7 +333,7 @@ private Response genericError(Response.Status status, String code, String msg, S { return Response .status(status) - .entity(new SqlTaskStatus(id, TaskState.FAILED, new QueryException("FAILED", msg, null, null))) + .entity(new SqlTaskStatus(id, TaskState.FAILED, new QueryException("FAILED", msg, null, null, null))) .build(); } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQControllerTaskTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQControllerTaskTest.java index e1c5c8bf53db..33e0d8c24869 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQControllerTaskTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQControllerTaskTest.java @@ -65,6 +65,7 @@ public void testGetInputSourceResources() null, null, null, + null, null); Assert.assertTrue(msqWorkerTask.getInputSourceResources().isEmpty()); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java index 99390da07dcf..f27746ac7332 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java @@ -24,6 +24,8 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.druid.msq.indexing.MSQControllerTask; +import org.apache.druid.msq.indexing.error.InsertCannotBeEmptyFault; import org.apache.druid.msq.sql.entity.ColNameAndType; import org.apache.druid.msq.sql.entity.ResultSetInformation; import org.apache.druid.msq.sql.entity.SqlStatementResult; @@ -32,6 +34,8 @@ import org.apache.druid.msq.test.MSQTestOverlordServiceClient; import org.apache.druid.query.ExecutionMode; import org.apache.druid.query.QueryContexts; +import org.apache.druid.query.QueryException; +import org.apache.druid.segment.column.ValueType; import org.apache.druid.server.initialization.ServerConfig; import org.apache.druid.sql.calcite.util.CalciteTests; import org.apache.druid.sql.http.SqlQuery; @@ -81,8 +85,7 @@ public void testMSQSelectQueryTest() throws IOException false, false, false, - ImmutableMap.of( - QueryContexts.CTX_EXECUTION_MODE, ExecutionMode.ASYNC.name()), + ImmutableMap.of(QueryContexts.CTX_EXECUTION_MODE, ExecutionMode.ASYNC.name()), null ), SqlStatementResourceTest.makeOkRequest()); @@ -91,24 +94,27 @@ public void testMSQSelectQueryTest() throws IOException String taskId = ((SqlStatementResult) response.getEntity()).getQueryId(); - SqlStatementResult sqlStatementResult = + SqlStatementResult expected = new SqlStatementResult(taskId, SqlStatementState.SUCCESS, MSQTestOverlordServiceClient.CREATED_TIME, ImmutableList.of( new ColNameAndType( "cnt", - SqlTypeName.BIGINT.getName() + SqlTypeName.BIGINT.getName(), + ValueType.LONG.name() ), new ColNameAndType( "dim1", - SqlTypeName.VARCHAR.getName() + SqlTypeName.VARCHAR.getName(), + ValueType.STRING.name() ) ), MSQTestOverlordServiceClient.DURATION, new ResultSetInformation( null, 6L, - null, + 316L, + MSQControllerTask.DUMMY_DATASOURCE_FOR_SELECT, objectMapper.readValue( objectMapper.writeValueAsString( results), @@ -120,9 +126,49 @@ public void testMSQSelectQueryTest() throws IOException null ); - Assert.assertEquals(sqlStatementResult, response.getEntity()); + Assert.assertEquals(expected, response.getEntity()); + } + + + @Test + public void insertCannotBeEmptyFaultTest() + { + Response response = resource.doPost(new SqlQuery( + "insert into foo1 select __time, dim1 , count(*) as cnt from foo where dim1 is not null and __time < TIMESTAMP '1971-01-01 00:00:00' group by 1, 2 PARTITIONED by day clustered by dim1", + null, + false, + false, + false, + ImmutableMap.of(QueryContexts.CTX_EXECUTION_MODE, ExecutionMode.ASYNC.name()), + null + ), SqlStatementResourceTest.makeOkRequest()); + Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); + + SqlStatementResult actual = (SqlStatementResult) response.getEntity(); + InsertCannotBeEmptyFault insertCannotBeEmptyFault = new InsertCannotBeEmptyFault("foo1"); + SqlStatementResult expected = new SqlStatementResult( + actual.getQueryId(), + SqlStatementState.FAILED, + MSQTestOverlordServiceClient.CREATED_TIME, + null, + MSQTestOverlordServiceClient.DURATION, + null, + new QueryException( + null, + insertCannotBeEmptyFault.getErrorMessage(), + null, + "localhost:8080", + ImmutableMap.of( + "errorCode", + InsertCannotBeEmptyFault.CODE, + "dataSource", + insertCannotBeEmptyFault.getDataSource() + ) + ) + ); + Assert.assertEquals(expected, actual); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java index 08d3652e15df..427ceff4c0bf 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java @@ -67,6 +67,7 @@ import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.column.ValueType; import org.apache.druid.server.initialization.ServerConfig; import org.apache.druid.server.mocks.MockHttpServletRequest; import org.apache.druid.server.security.AuthConfig; @@ -160,6 +161,7 @@ public class SqlStatementResourceTest extends MSQTestBase Maps.newHashMap(), null, ImmutableList.of(SqlTypeName.TIMESTAMP, SqlTypeName.VARCHAR, SqlTypeName.VARCHAR), + ImmutableList.of(ColumnType.LONG, ColumnType.STRING, ColumnType.STRING), null ); @@ -192,6 +194,7 @@ public class SqlStatementResourceTest extends MSQTestBase Maps.newHashMap(), null, ImmutableList.of(SqlTypeName.TIMESTAMP, SqlTypeName.VARCHAR, SqlTypeName.VARCHAR), + ImmutableList.of(ColumnType.LONG, ColumnType.STRING, ColumnType.STRING), null ); @@ -250,6 +253,12 @@ public class SqlStatementResourceTest extends MSQTestBase private static final Map ROW1 = ImmutableMap.of("_time", 123, "alias", "foo", "market", "bar"); private static final Map ROW2 = ImmutableMap.of("_time", 234, "alias", "foo1", "market", "bar1"); + private static final ImmutableList COL_NAME_AND_TYPES = ImmutableList.of( + new ColNameAndType("_time", SqlTypeName.TIMESTAMP.getName(), ValueType.LONG.name()), + new ColNameAndType("alias", SqlTypeName.VARCHAR.getName(), ValueType.STRING.name()), + new ColNameAndType("market", SqlTypeName.VARCHAR.getName(), ValueType.STRING.name()) + ); + private static final String FAILURE_MSG = "failure msg"; @Before @@ -541,11 +550,7 @@ public void testMSQSelectAcceptedQuery() ACCEPTED_SELECT_MSQ_QUERY, SqlStatementState.ACCEPTED, CREATED_TIME, - ImmutableList.of( - new ColNameAndType("_time", SqlTypeName.TIMESTAMP.getName()), - new ColNameAndType("alias", SqlTypeName.VARCHAR.getName()), - new ColNameAndType("market", SqlTypeName.VARCHAR.getName()) - ), + COL_NAME_AND_TYPES, null, null, null @@ -573,11 +578,7 @@ public void testMSQSelectRunningQuery() RUNNING_SELECT_MSQ_QUERY, SqlStatementState.RUNNING, CREATED_TIME, - ImmutableList.of( - new ColNameAndType("_time", SqlTypeName.TIMESTAMP.getName()), - new ColNameAndType("alias", SqlTypeName.VARCHAR.getName()), - new ColNameAndType("market", SqlTypeName.VARCHAR.getName()) - ), + COL_NAME_AND_TYPES, null, null, null @@ -604,16 +605,13 @@ public void testFinishedSelectMSQQuery() throws Exception FINISHED_SELECT_MSQ_QUERY, SqlStatementState.SUCCESS, CREATED_TIME, - ImmutableList.of( - new ColNameAndType("_time", SqlTypeName.TIMESTAMP.getName()), - new ColNameAndType("alias", SqlTypeName.VARCHAR.getName()), - new ColNameAndType("market", SqlTypeName.VARCHAR.getName()) - ), + COL_NAME_AND_TYPES, 100L, new ResultSetInformation( null, 2L, null, + null, RESULT_ROWS.stream().map(Arrays::asList).collect(Collectors.toList()) ), null diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlTaskStatusTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlTaskStatusTest.java index 03efc96e6de4..dccb19344153 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlTaskStatusTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlTaskStatusTest.java @@ -41,7 +41,8 @@ public void testSerde() throws Exception "error code", "error message", "error class", - "host" + "host", + null ) ); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestOverlordServiceClient.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestOverlordServiceClient.java index 8494d6014718..1b49982cad46 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestOverlordServiceClient.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestOverlordServiceClient.java @@ -28,8 +28,7 @@ import org.apache.druid.client.indexing.NoopOverlordClient; import org.apache.druid.client.indexing.TaskPayloadResponse; import org.apache.druid.client.indexing.TaskStatusResponse; -import org.apache.druid.indexer.TaskLocation; -import org.apache.druid.indexer.TaskState; +import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.indexing.common.TaskReport; import org.apache.druid.indexing.common.actions.TaskActionClient; @@ -55,6 +54,7 @@ public class MSQTestOverlordServiceClient extends NoopOverlordClient private Map inMemoryControllers = new HashMap<>(); private Map> reports = new HashMap<>(); private Map inMemoryControllerTask = new HashMap<>(); + private Map inMemoryTaskStatus = new HashMap<>(); public static final DateTime CREATED_TIME = DateTimes.of("2023-05-31T12:00Z"); public static final DateTime QUEUE_INSERTION_TIME = DateTimes.of("2023-05-31T12:01Z"); @@ -94,7 +94,7 @@ public ListenableFuture runTask(String taskId, Object taskObject) inMemoryControllers.put(controller.id(), controller); - controller.run(); + inMemoryTaskStatus.put(taskId, controller.run()); return Futures.immediateFuture(null); } catch (Exception e) { @@ -145,18 +145,19 @@ public ListenableFuture taskPayload(String taskId) public ListenableFuture taskStatus(String taskId) { SettableFuture future = SettableFuture.create(); + TaskStatus taskStatus = inMemoryTaskStatus.get(taskId); future.set(new TaskStatusResponse(taskId, new TaskStatusPlus( taskId, null, MSQControllerTask.TYPE, CREATED_TIME, QUEUE_INSERTION_TIME, - TaskState.SUCCESS, + taskStatus.getStatusCode(), null, DURATION, - new TaskLocation("localhost", 1, 1), + taskStatus.getLocation(), null, - null + taskStatus.getErrorMsg() ))); return future; diff --git a/processing/src/main/java/org/apache/druid/query/BadQueryException.java b/processing/src/main/java/org/apache/druid/query/BadQueryException.java index e627c966ade8..607950bf884a 100644 --- a/processing/src/main/java/org/apache/druid/query/BadQueryException.java +++ b/processing/src/main/java/org/apache/druid/query/BadQueryException.java @@ -40,6 +40,6 @@ protected BadQueryException(String errorCode, String errorMessage, String errorC protected BadQueryException(Throwable cause, String errorCode, String errorMessage, String errorClass, String host) { - super(cause, errorCode, errorMessage, errorClass, host); + super(cause, errorCode, errorMessage, errorClass, host, null); } } diff --git a/processing/src/main/java/org/apache/druid/query/QueryCapacityExceededException.java b/processing/src/main/java/org/apache/druid/query/QueryCapacityExceededException.java index 694fbb780cb6..2229e1fe9135 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryCapacityExceededException.java +++ b/processing/src/main/java/org/apache/druid/query/QueryCapacityExceededException.java @@ -47,12 +47,12 @@ public class QueryCapacityExceededException extends QueryException public QueryCapacityExceededException(int capacity) { - super(QUERY_CAPACITY_EXCEEDED_ERROR_CODE, makeTotalErrorMessage(capacity), ERROR_CLASS, null); + super(QUERY_CAPACITY_EXCEEDED_ERROR_CODE, makeTotalErrorMessage(capacity), ERROR_CLASS, null, null); } public QueryCapacityExceededException(String lane, int capacity) { - super(QUERY_CAPACITY_EXCEEDED_ERROR_CODE, makeLaneErrorMessage(lane, capacity), ERROR_CLASS, null); + super(QUERY_CAPACITY_EXCEEDED_ERROR_CODE, makeLaneErrorMessage(lane, capacity), ERROR_CLASS, null, null); } /** @@ -77,7 +77,7 @@ public QueryCapacityExceededException( @JsonProperty("host") String host ) { - super(errorCode, errorMessage, errorClass, host); + super(errorCode, errorMessage, errorClass, host, null); } @VisibleForTesting diff --git a/processing/src/main/java/org/apache/druid/query/QueryException.java b/processing/src/main/java/org/apache/druid/query/QueryException.java index 5dbdef6ddb7e..75634e457637 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryException.java +++ b/processing/src/main/java/org/apache/druid/query/QueryException.java @@ -20,13 +20,17 @@ package org.apache.druid.query; import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Joiner; import org.apache.druid.common.exception.SanitizableException; import org.apache.druid.java.util.common.StringUtils; import javax.annotation.Nullable; import javax.validation.constraints.NotNull; import java.net.InetAddress; +import java.util.Map; +import java.util.Objects; import java.util.function.Function; /** @@ -151,17 +155,27 @@ public static FailType fromErrorCode(String errorCode) private final String errorClass; private final String host; + private final Map details; + protected QueryException(Throwable cause, String errorCode, String errorClass, String host) { - this(cause, errorCode, cause == null ? null : cause.getMessage(), errorClass, host); + this(cause, errorCode, cause == null ? null : cause.getMessage(), errorClass, host, null); } - protected QueryException(Throwable cause, String errorCode, String errorMessage, String errorClass, String host) + protected QueryException( + Throwable cause, + String errorCode, + String errorMessage, + String errorClass, + String host, + Map details + ) { super(errorMessage, cause); this.errorCode = errorCode; this.errorClass = errorClass; this.host = host; + this.details = details; } @JsonCreator @@ -169,13 +183,15 @@ public QueryException( @JsonProperty("error") @Nullable String errorCode, @JsonProperty("errorMessage") String errorMessage, @JsonProperty("errorClass") @Nullable String errorClass, - @JsonProperty("host") @Nullable String host + @JsonProperty("host") @Nullable String host, + @JsonProperty("details") @Nullable Map details ) { super(errorMessage); this.errorCode = errorCode; this.errorClass = errorClass; this.host = host; + this.details = details; } @Nullable @@ -204,6 +220,14 @@ public String getHost() return host; } + @JsonProperty + @Nullable + @JsonInclude(JsonInclude.Include.NON_NULL) + public Map getDetails() + { + return details; + } + @Nullable protected static String resolveHostname() { @@ -218,7 +242,7 @@ protected static String resolveHostname() @Override public QueryException sanitize(@NotNull Function errorMessageTransformFunction) { - return new QueryException(errorCode, errorMessageTransformFunction.apply(getMessage()), null, null); + return new QueryException(errorCode, errorMessageTransformFunction.apply(getMessage()), null, null, null); } public FailType getFailType() @@ -230,12 +254,35 @@ public FailType getFailType() public String toString() { return StringUtils.format( - "%s{msg=%s, code=%s, class=%s, host=%s}", + "%s{msg=%s, code=%s, class=%s, host=%s, details=%s}", getClass().getSimpleName(), getMessage(), getErrorCode(), getErrorClass(), - getHost() + getHost(), + getDetails() != null ? Joiner.on(",").withKeyValueSeparator("=").join(getDetails()) : null ); } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + QueryException that = (QueryException) o; + return Objects.equals(errorCode, that.errorCode) + && Objects.equals(errorClass, that.errorClass) + && Objects.equals(host, that.host) + && Objects.equals(details, that.details); + } + + @Override + public int hashCode() + { + return Objects.hash(errorCode, errorClass, host, details); + } } diff --git a/processing/src/main/java/org/apache/druid/query/QueryInterruptedException.java b/processing/src/main/java/org/apache/druid/query/QueryInterruptedException.java index f1893cf3eaaa..8c93bc13b75a 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryInterruptedException.java +++ b/processing/src/main/java/org/apache/druid/query/QueryInterruptedException.java @@ -50,7 +50,7 @@ public QueryInterruptedException( @JsonProperty("host") @Nullable String host ) { - super(errorCode, errorMessage, errorClass, host); + super(errorCode, errorMessage, errorClass, host, null); } /** diff --git a/processing/src/main/java/org/apache/druid/query/QueryTimeoutException.java b/processing/src/main/java/org/apache/druid/query/QueryTimeoutException.java index 84be0a5ca37a..af1290be4f55 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryTimeoutException.java +++ b/processing/src/main/java/org/apache/druid/query/QueryTimeoutException.java @@ -48,21 +48,21 @@ public QueryTimeoutException( @JsonProperty("host") @Nullable String host ) { - super(errorCode, errorMessage, errorClass, host); + super(errorCode, errorMessage, errorClass, host, null); } public QueryTimeoutException() { - super(QUERY_TIMEOUT_ERROR_CODE, ERROR_MESSAGE, ERROR_CLASS, resolveHostname()); + super(QUERY_TIMEOUT_ERROR_CODE, ERROR_MESSAGE, ERROR_CLASS, resolveHostname(), null); } public QueryTimeoutException(String errorMessage) { - super(QUERY_TIMEOUT_ERROR_CODE, errorMessage, ERROR_CLASS, resolveHostname()); + super(QUERY_TIMEOUT_ERROR_CODE, errorMessage, ERROR_CLASS, resolveHostname(), null); } public QueryTimeoutException(String errorMessage, String host) { - super(QUERY_TIMEOUT_ERROR_CODE, errorMessage, ERROR_CLASS, host); + super(QUERY_TIMEOUT_ERROR_CODE, errorMessage, ERROR_CLASS, host, null); } } diff --git a/processing/src/main/java/org/apache/druid/query/QueryUnsupportedException.java b/processing/src/main/java/org/apache/druid/query/QueryUnsupportedException.java index 81d82a94871a..81d69ea4849a 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryUnsupportedException.java +++ b/processing/src/main/java/org/apache/druid/query/QueryUnsupportedException.java @@ -46,11 +46,11 @@ public QueryUnsupportedException( @JsonProperty("host") @Nullable String host ) { - super(errorCode, errorMessage, errorClass, host); + super(errorCode, errorMessage, errorClass, host, null); } public QueryUnsupportedException(String errorMessage) { - super(QUERY_UNSUPPORTED_ERROR_CODE, errorMessage, ERROR_CLASS, resolveHostname()); + super(QUERY_UNSUPPORTED_ERROR_CODE, errorMessage, ERROR_CLASS, resolveHostname(), null); } } diff --git a/processing/src/test/java/org/apache/druid/query/QueryExceptionTest.java b/processing/src/test/java/org/apache/druid/query/QueryExceptionTest.java index 10a0b78521da..c49feb532e7e 100644 --- a/processing/src/test/java/org/apache/druid/query/QueryExceptionTest.java +++ b/processing/src/test/java/org/apache/druid/query/QueryExceptionTest.java @@ -36,7 +36,7 @@ public class QueryExceptionTest @Test public void testSanitizeWithTransformFunctionReturningNull() { - QueryException queryException = new QueryException(ERROR_CODE, ERROR_MESSAGE_ORIGINAL, ERROR_CLASS, HOST); + QueryException queryException = new QueryException(ERROR_CODE, ERROR_MESSAGE_ORIGINAL, ERROR_CLASS, HOST, null); AtomicLong callCount = new AtomicLong(0); QueryException actual = queryException.sanitize(s -> { @@ -56,7 +56,7 @@ public void testSanitizeWithTransformFunctionReturningNull() @Test public void testSanitizeWithTransformFunctionReturningNewString() { - QueryException queryException = new QueryException(ERROR_CODE, ERROR_MESSAGE_ORIGINAL, ERROR_CLASS, HOST); + QueryException queryException = new QueryException(ERROR_CODE, ERROR_MESSAGE_ORIGINAL, ERROR_CLASS, HOST, null); AtomicLong callCount = new AtomicLong(0); QueryException actual = queryException.sanitize(s -> { @@ -116,7 +116,7 @@ public void testCanConstructWithoutThrowable() @Test public void testToStringReturnsUsefulInformation() { - QueryException queryException = new QueryException(ERROR_CODE, ERROR_MESSAGE_ORIGINAL, ERROR_CLASS, HOST); + QueryException queryException = new QueryException(ERROR_CODE, ERROR_MESSAGE_ORIGINAL, ERROR_CLASS, HOST, null); String exceptionToString = queryException.toString(); Assert.assertTrue(exceptionToString.startsWith(QueryException.class.getSimpleName())); Assert.assertTrue(exceptionToString.contains("msg=" + ERROR_MESSAGE_ORIGINAL)); diff --git a/server/src/main/java/org/apache/druid/server/QueryResource.java b/server/src/main/java/org/apache/druid/server/QueryResource.java index f4a7ab3edb75..5a82c317ac6e 100644 --- a/server/src/main/java/org/apache/druid/server/QueryResource.java +++ b/server/src/main/java/org/apache/druid/server/QueryResource.java @@ -241,7 +241,8 @@ public Response doPost( QueryException.UNKNOWN_EXCEPTION_ERROR_CODE, "Unhandled exception made it to the top", e.getClass().getName(), - req.getRemoteHost() + req.getRemoteHost(), + null ); out.write(jsonMapper.writeValueAsBytes(responseException)); } diff --git a/server/src/test/java/org/apache/druid/client/JsonParserIteratorTest.java b/server/src/test/java/org/apache/druid/client/JsonParserIteratorTest.java index f896067646cc..1f0683f0456e 100644 --- a/server/src/test/java/org/apache/druid/client/JsonParserIteratorTest.java +++ b/server/src/test/java/org/apache/druid/client/JsonParserIteratorTest.java @@ -78,7 +78,8 @@ public void testConvertFutureTimeoutToQueryTimeoutException() QueryException.QUERY_TIMEOUT_ERROR_CODE, "timeout exception conversion test", null, - HOST + HOST, + null ) ), URL, @@ -201,7 +202,13 @@ public void testConvertQueryExceptionWithNullErrorCodeToQueryInterruptedExceptio { JsonParserIterator iterator = new JsonParserIterator<>( JAVA_TYPE, - Futures.immediateFuture(mockErrorResponse(new QueryException(null, "query exception test", null, null))), + Futures.immediateFuture(mockErrorResponse(new QueryException( + null, + "query exception test", + null, + null, + null + ))), URL, null, HOST, @@ -219,7 +226,7 @@ public void testConvertQueryExceptionWithNonNullErrorCodeToQueryInterruptedExcep JsonParserIterator iterator = new JsonParserIterator<>( JAVA_TYPE, Futures.immediateFuture( - mockErrorResponse(new QueryException("test error", "query exception test", null, null)) + mockErrorResponse(new QueryException("test error", "query exception test", null, null, null)) ), URL, null, diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java index af5983d6edbf..4e7313c08949 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java @@ -323,6 +323,7 @@ public ArrayList getOrdering() QueryException.SQL_QUERY_UNSUPPORTED_ERROR_CODE, StringUtils.format("Cannot handle ordering with direction[%s]", field.direction), null, + null, null ); } diff --git a/sql/src/test/java/org/apache/druid/sql/avatica/ErrorHandlerTest.java b/sql/src/test/java/org/apache/druid/sql/avatica/ErrorHandlerTest.java index 2efa65d1a7b5..158d17a441bc 100644 --- a/sql/src/test/java/org/apache/druid/sql/avatica/ErrorHandlerTest.java +++ b/sql/src/test/java/org/apache/druid/sql/avatica/ErrorHandlerTest.java @@ -41,7 +41,7 @@ public void testErrorHandlerSanitizesErrorAsExpected() Mockito.when(serverConfig.getErrorResponseTransformStrategy()) .thenReturn(emptyAllowedRegexErrorResponseTransformStrategy); ErrorHandler errorHandler = new ErrorHandler(serverConfig); - QueryException input = new QueryException("error", "error message", "error class", "host"); + QueryException input = new QueryException("error", "error message", "error class", "host", null); RuntimeException output = errorHandler.sanitize(input); Assert.assertNull(output.getMessage()); From 8006f6d2990a912907824c30c0ba63645eb946e7 Mon Sep 17 00:00:00 2001 From: cryptoe Date: Wed, 21 Jun 2023 18:05:26 +0530 Subject: [PATCH 05/15] Fixing checkstyle --- .../msq/sql/resources/SqlStatementResource.java | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java index 959492f18500..c192bba75051 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java @@ -153,9 +153,7 @@ public Response doGetEnabled(@Context final HttpServletRequest request) @POST @Produces(MediaType.APPLICATION_JSON) @Consumes(MediaType.APPLICATION_JSON) - public Response doPost( - final SqlQuery sqlQuery, @Context final HttpServletRequest req - ) + public Response doPost(final SqlQuery sqlQuery, @Context final HttpServletRequest req) { final HttpStatement stmt = msqSqlStatementFactory.httpStatement(sqlQuery, req); final String sqlQueryId = stmt.sqlQueryId(); @@ -460,7 +458,10 @@ public Response deleteQuery(@PathParam("id") final String queryId, @Context fina } private Response buildStandardResponse( - Sequence sequence, SqlQuery sqlQuery, String sqlQueryId, SqlRowTransformer rowTransformer + Sequence sequence, + SqlQuery sqlQuery, + String sqlQueryId, + SqlRowTransformer rowTransformer ) throws IOException { final Yielder yielder0 = Yielders.each(sequence); @@ -621,10 +622,12 @@ private void checkTaskPayloadOrThrow(TaskPayloadResponse taskPayloadResponse, St } private Optional getSampleResults( - String asyncResultId, boolean isSelectQuery, String dataSource, SqlStatementState sqlStatementState + String asyncResultId, + boolean isSelectQuery, + String dataSource, + SqlStatementState sqlStatementState ) { - if (sqlStatementState == SqlStatementState.SUCCESS) { Map payload = getPayload(overlordWork(overlordClient.taskReportAsMap(asyncResultId))); Optional> rowsAndSize = getRowsAndSizeFromPayload(payload, isSelectQuery); From 4bdd18a0141e52f73d0b3b3d6f173cf77fda8c5d Mon Sep 17 00:00:00 2001 From: cryptoe Date: Thu, 22 Jun 2023 07:30:22 +0530 Subject: [PATCH 06/15] Merge with main branch. --- .../sql/resources/SqlStatementResource.java | 3 ++- .../msq/sql/SqlStatementResourceTest.java | 25 ++++++++++++++++--- .../org/apache/druid/error/ErrorResponse.java | 3 ++- 3 files changed, 26 insertions(+), 5 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java index c192bba75051..6ebd3f26ce7c 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java @@ -701,6 +701,8 @@ private Optional getStatementStatus(String queryId, String c return Optional.empty(); } + // since we need the controller payload for auth checks. + MSQControllerTask msqControllerTask = getMSQControllerTaskOrThrow(queryId, currentUser); SqlStatementState sqlStatementState = getSqlStatementState(statusPlus); if (SqlStatementState.FAILED == sqlStatementState) { @@ -736,7 +738,6 @@ private Optional getStatementStatus(String queryId, String c new QueryException(null, errorMessage, null, host, stringException) )); } else { - MSQControllerTask msqControllerTask = getMSQControllerTaskOrThrow(queryId, currentUser); Optional> signature = getSignature(msqControllerTask); return Optional.of(new SqlStatementResult( diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java index 427ceff4c0bf..00e3f4b66810 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java @@ -222,7 +222,8 @@ public class SqlStatementResourceTest extends MSQTestBase new MSQResultsReport.ColumnAndType("market", ColumnType.STRING) ), ImmutableList.of(SqlTypeName.TIMESTAMP, SqlTypeName.VARCHAR, SqlTypeName.VARCHAR), - Yielders.each(Sequences.simple(RESULT_ROWS)) + Yielders.each(Sequences.simple(RESULT_ROWS)), + null ) ) ); @@ -383,6 +384,14 @@ private static void setupMocks(OverlordClient indexingServiceClient) throws Json ) ))); + Mockito.when(indexingServiceClient.taskPayload(ArgumentMatchers.eq(ERRORED_SELECT_MSQ_QUERY))) + .thenReturn(Futures.immediateFuture(new TaskPayloadResponse( + FINISHED_INSERT_MSQ_QUERY, + MSQ_CONTROLLER_SELECT_PAYLOAD + ))); + + Mockito.when(indexingServiceClient.taskReportAsMap(ArgumentMatchers.eq(ERRORED_SELECT_MSQ_QUERY))) + .thenReturn(Futures.immediateFuture(null)); Mockito.when(indexingServiceClient.taskStatus(ArgumentMatchers.eq(RUNNING_NON_MSQ_TASK))) .thenReturn(Futures.immediateFuture(new TaskStatusResponse( @@ -539,6 +548,16 @@ private static void setupMocks(OverlordClient indexingServiceClient) throws Json FAILURE_MSG ) ))); + + Mockito.when(indexingServiceClient.taskPayload(ArgumentMatchers.eq(ERRORED_INSERT_MSQ_QUERY))) + .thenReturn(Futures.immediateFuture(new TaskPayloadResponse( + ERRORED_INSERT_MSQ_QUERY, + MSQ_CONTROLLER_INSERT_PAYLOAD + ))); + + Mockito.when(indexingServiceClient.taskReportAsMap(ArgumentMatchers.eq(ERRORED_INSERT_MSQ_QUERY))) + .thenReturn(Futures.immediateFuture(null)); + } @Test @@ -609,9 +628,9 @@ public void testFinishedSelectMSQQuery() throws Exception 100L, new ResultSetInformation( null, - 2L, null, null, + MSQControllerTask.DUMMY_DATASOURCE_FOR_SELECT, RESULT_ROWS.stream().map(Arrays::asList).collect(Collectors.toList()) ), null @@ -721,7 +740,7 @@ public void testFinishedInsertMSQQuery() throws Exception CREATED_TIME, null, 100L, - null, + new ResultSetInformation(null, null, null, "test", null), null ), response.getEntity()); diff --git a/processing/src/main/java/org/apache/druid/error/ErrorResponse.java b/processing/src/main/java/org/apache/druid/error/ErrorResponse.java index 7b571cca2719..2a89dc662687 100644 --- a/processing/src/main/java/org/apache/druid/error/ErrorResponse.java +++ b/processing/src/main/java/org/apache/druid/error/ErrorResponse.java @@ -79,7 +79,8 @@ public static ErrorResponse fromMap(Map map) nullOrString(map.get("error")), nullOrString(map.get("errorMessage")), nullOrString(map.get("errorClass")), - nullOrString(map.get("host")) + nullOrString(map.get("host")), + null ) ); } else { From c6d3df3812c4fd87f3b4aa7e29b506b99706b893 Mon Sep 17 00:00:00 2001 From: cryptoe Date: Sat, 24 Jun 2023 06:44:42 +0530 Subject: [PATCH 07/15] Cleaning up phase 1. --- .../druid/msq/sql/SqlStatementState.java | 14 + ...meAndType.java => ColumnNameAndTypes.java} | 11 +- .../msq/sql/entity/SqlStatementResult.java | 6 +- .../sql/resources/SqlStatementResource.java | 30 +- .../sql/SqlMsqStatementResourcePostTest.java | 6 +- .../msq/sql/SqlStatementResourceTest.java | 650 +++++++++--------- .../apache/druid/error/DruidException.java | 41 +- .../org/apache/druid/sql/DirectStatement.java | 4 +- 8 files changed, 394 insertions(+), 368 deletions(-) rename extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/{ColNameAndType.java => ColumnNameAndTypes.java} (88%) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlStatementState.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlStatementState.java index 6b45272394d2..e7f7da95d8e7 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlStatementState.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlStatementState.java @@ -19,10 +19,24 @@ package org.apache.druid.msq.sql; +import org.apache.druid.sql.http.SqlQuery; + +import javax.servlet.http.HttpServletRequest; + +/** + * Represents the status of the sql statements issues via + * {@link org.apache.druid.msq.sql.resources.SqlStatementResource#doPost(SqlQuery, HttpServletRequest)} and returned in + * {@link org.apache.druid.msq.sql.entity.SqlStatementResult} + */ public enum SqlStatementState { + // The statement is accepted but not yes assigned any worker. In MSQ engine, the statement is in ACCEPTED state + // till the overlord assigns a TaskLocation to the controller task. ACCEPTED, + // The statement is running. RUNNING, + // The statement is successful. SUCCESS, + // The statement failed. FAILED } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ColNameAndType.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ColumnNameAndTypes.java similarity index 88% rename from extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ColNameAndType.java rename to extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ColumnNameAndTypes.java index 73b9d246aa8a..3f7188c5d93d 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ColNameAndType.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ColumnNameAndTypes.java @@ -24,8 +24,11 @@ import java.util.Objects; +/** + * The column name and its sql {@link org.apache.calcite.sql.type.SqlTypeName} and its native {@link org.apache.druid.segment.column.ColumnType} + */ -public class ColNameAndType +public class ColumnNameAndTypes { private final String colName; @@ -34,7 +37,7 @@ public class ColNameAndType private final String nativeTypeName; @JsonCreator - public ColNameAndType( + public ColumnNameAndTypes( @JsonProperty("name") String colName, @JsonProperty("type") String sqlTypeName, @JsonProperty("nativeType") String nativeTypeName @@ -73,7 +76,7 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - ColNameAndType that = (ColNameAndType) o; + ColumnNameAndTypes that = (ColumnNameAndTypes) o; return Objects.equals(colName, that.colName) && Objects.equals(sqlTypeName, that.sqlTypeName) && Objects.equals(nativeTypeName, that.nativeTypeName); @@ -88,7 +91,7 @@ public int hashCode() @Override public String toString() { - return "ColNameAndType{" + + return "ColumnNameAndTypes{" + "colName='" + colName + '\'' + ", sqlTypeName='" + sqlTypeName + '\'' + ", nativeTypeName='" + nativeTypeName + '\'' + diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/SqlStatementResult.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/SqlStatementResult.java index 1d4c6c9d726b..4d6ba7099658 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/SqlStatementResult.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/SqlStatementResult.java @@ -40,7 +40,7 @@ public class SqlStatementResult private final DateTime createdAt; @Nullable - private final List sqlRowSignature; + private final List sqlRowSignature; @Nullable private final Long durationInMs; @@ -61,7 +61,7 @@ public SqlStatementResult( @JsonProperty("createdAt") DateTime createdAt, @Nullable @JsonProperty("schema") - List sqlRowSignature, + List sqlRowSignature, @Nullable @JsonProperty("durationMs") Long durationMs, @Nullable @JsonProperty("result") @@ -101,7 +101,7 @@ public DateTime getCreatedAt() @JsonProperty("schema") @Nullable @JsonInclude(JsonInclude.Include.NON_NULL) - public List getSqlRowSignature() + public List getSqlRowSignature() { return sqlRowSignature; } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java index 6ebd3f26ce7c..f33ec03391a4 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java @@ -48,7 +48,7 @@ import org.apache.druid.msq.sql.MSQTaskQueryMaker; import org.apache.druid.msq.sql.MSQTaskSqlEngine; import org.apache.druid.msq.sql.SqlStatementState; -import org.apache.druid.msq.sql.entity.ColNameAndType; +import org.apache.druid.msq.sql.entity.ColumnNameAndTypes; import org.apache.druid.msq.sql.entity.ResultSetInformation; import org.apache.druid.msq.sql.entity.SqlStatementResult; import org.apache.druid.query.BadQueryException; @@ -134,7 +134,7 @@ public SqlStatementResource( @GET @Path("/enabled") @Produces(MediaType.APPLICATION_JSON) - public Response doGetEnabled(@Context final HttpServletRequest request) + public Response isEnabled(@Context final HttpServletRequest request) { // All authenticated users are authorized for this API: check an empty resource list. final Access authResult = AuthorizationUtils.authorizeAllResourceActions( @@ -187,6 +187,7 @@ public Response doPost(final SqlQuery sqlQuery, @Context final HttpServletReques Thread.currentThread().setName(StringUtils.format("statement_sql[%s]", sqlQueryId)); final DirectStatement.ResultSet plan = stmt.plan(); + // in case the engine is async, the query is not run yet. We just return the taskID in case of non explain queries. final QueryResponse response = plan.run(); final Sequence sequence = response.getResults(); final SqlRowTransformer rowTransformer = plan.createRowTransformer(); @@ -290,8 +291,6 @@ public Response doGetResults( @PathParam("id") final String queryId, @QueryParam("offset") Long offset, @QueryParam("numRows") Long numberOfRows, - @QueryParam("sizeInBytes") Long size, - @QueryParam("timeout") Integer timeout, @Context final HttpServletRequest req ) { @@ -359,7 +358,7 @@ public Response doGetResults( ); } else { MSQControllerTask msqControllerTask = getMSQControllerTaskOrThrow(queryId, authenticationResult.getIdentity()); - Optional> signature = getSignature(msqControllerTask); + Optional> signature = getSignature(msqControllerTask); if (!signature.isPresent()) { return Response.ok().build(); } @@ -369,13 +368,16 @@ public Response doGetResults( CountingOutputStream os = new CountingOutputStream(outputStream); try (final ResultFormat.Writer writer = ResultFormat.OBJECT.createFormatter(os, jsonMapper)) { - List rowSignature = signature.get(); + List rowSignature = signature.get(); writer.writeResponseStart(); - for (int k = (int) start; k < Math.min(last, results.get().size()); k++) { + for (long k = start; k < Math.min(last, results.get().size()); k++) { writer.writeRowStart(); for (int i = 0; i < rowSignature.size(); i++) { - writer.writeRowField(rowSignature.get(i).getColName(), ((List) results.get().get(k)).get(i)); + writer.writeRowField( + rowSignature.get(i).getColName(), + ((List) results.get().get(Math.toIntExact(k))).get(i) + ); } writer.writeRowEnd(); } @@ -561,7 +563,7 @@ private Response buildNonOkResponse(int status, SanitizableException e, String s } - private static Optional> getSignature( + private static Optional> getSignature( MSQControllerTask msqControllerTask ) { @@ -576,10 +578,10 @@ private static Optional> getSignature( if (nativeTypeNames == null || nativeTypeNames.size() != columnMappings.size()) { return Optional.empty(); } - List signature = new ArrayList<>(columnMappings.size()); + List signature = new ArrayList<>(columnMappings.size()); int index = 0; for (String colName : columnMappings.getOutputColumnNames()) { - signature.add(new ColNameAndType( + signature.add(new ColumnNameAndTypes( colName, sqlTypeNames.get(index).getName(), nativeTypeNames.get(index).asTypeString() @@ -653,8 +655,8 @@ private Optional> getRowsAndSizeFromPayload(Map } else { int maxStage = stages.size() - 1; // Last stage output is the total number of rows returned to the end user. Map counterMap = getMap(getMap(payload, "counters"), String.valueOf(maxStage)); - Long rows = -1L; - Long sizeInBytes = -1L; + long rows = -1L; + long sizeInBytes = -1L; if (counterMap == null) { return Optional.empty(); } @@ -738,7 +740,7 @@ private Optional getStatementStatus(String queryId, String c new QueryException(null, errorMessage, null, host, stringException) )); } else { - Optional> signature = getSignature(msqControllerTask); + Optional> signature = getSignature(msqControllerTask); return Optional.of(new SqlStatementResult( queryId, diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java index f27746ac7332..21948cbe0527 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java @@ -26,7 +26,7 @@ import org.apache.calcite.sql.type.SqlTypeName; import org.apache.druid.msq.indexing.MSQControllerTask; import org.apache.druid.msq.indexing.error.InsertCannotBeEmptyFault; -import org.apache.druid.msq.sql.entity.ColNameAndType; +import org.apache.druid.msq.sql.entity.ColumnNameAndTypes; import org.apache.druid.msq.sql.entity.ResultSetInformation; import org.apache.druid.msq.sql.entity.SqlStatementResult; import org.apache.druid.msq.sql.resources.SqlStatementResource; @@ -98,12 +98,12 @@ public void testMSQSelectQueryTest() throws IOException new SqlStatementResult(taskId, SqlStatementState.SUCCESS, MSQTestOverlordServiceClient.CREATED_TIME, ImmutableList.of( - new ColNameAndType( + new ColumnNameAndTypes( "cnt", SqlTypeName.BIGINT.getName(), ValueType.LONG.name() ), - new ColNameAndType( + new ColumnNameAndTypes( "dim1", SqlTypeName.VARCHAR.getName(), ValueType.STRING.name() diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java index 00e3f4b66810..db66baa24f73 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java @@ -53,7 +53,7 @@ import org.apache.druid.msq.indexing.report.MSQTaskReport; import org.apache.druid.msq.indexing.report.MSQTaskReportPayload; import org.apache.druid.msq.indexing.report.MSQTaskReportTest; -import org.apache.druid.msq.sql.entity.ColNameAndType; +import org.apache.druid.msq.sql.entity.ColumnNameAndTypes; import org.apache.druid.msq.sql.entity.ResultSetInformation; import org.apache.druid.msq.sql.entity.SqlStatementResult; import org.apache.druid.msq.sql.resources.SqlStatementResource; @@ -120,16 +120,17 @@ public class SqlStatementResourceTest extends MSQTestBase private static final String ERRORED_INSERT_MSQ_QUERY = "QUERY_ID_11"; - private static final Query QUERY = - new Druids.ScanQueryBuilder() - .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .legacy(false) - .intervals(new MultipleIntervalSegmentSpec( - Collections.singletonList(Intervals.of( - "2011-04-01T00:00:00.000Z/2011-04-03T00:00:00.000Z")))) - .dataSource("target") - .context(ImmutableMap.of(MSQTaskQueryMaker.USER_KEY, AuthConfig.ALLOW_ALL_NAME)) - .build(); + private static final Query QUERY = new Druids.ScanQueryBuilder().resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .legacy(false) + .intervals(new MultipleIntervalSegmentSpec( + Collections.singletonList(Intervals.of( + "2011-04-01T00:00:00.000Z/2011-04-03T00:00:00.000Z")))) + .dataSource("target") + .context(ImmutableMap.of( + MSQTaskQueryMaker.USER_KEY, + AuthConfig.ALLOW_ALL_NAME + )) + .build(); private static final MSQControllerTask MSQ_CONTROLLER_SELECT_PAYLOAD = new MSQControllerTask( @@ -160,8 +161,16 @@ public class SqlStatementResourceTest extends MSQTestBase "select _time,alias,market from test", Maps.newHashMap(), null, - ImmutableList.of(SqlTypeName.TIMESTAMP, SqlTypeName.VARCHAR, SqlTypeName.VARCHAR), - ImmutableList.of(ColumnType.LONG, ColumnType.STRING, ColumnType.STRING), + ImmutableList.of( + SqlTypeName.TIMESTAMP, + SqlTypeName.VARCHAR, + SqlTypeName.VARCHAR + ), + ImmutableList.of( + ColumnType.LONG, + ColumnType.STRING, + ColumnType.STRING + ), null ); @@ -185,16 +194,28 @@ public class SqlStatementResourceTest extends MSQTestBase ColumnType.STRING ) .build())) - .destination( - new DataSourceMSQDestination("test", Granularities.DAY, null, null)) + .destination(new DataSourceMSQDestination( + "test", + Granularities.DAY, + null, + null + )) .tuningConfig( MSQTuningConfig.defaultConfig()) .build(), "insert into test select _time,alias,market from test", Maps.newHashMap(), null, - ImmutableList.of(SqlTypeName.TIMESTAMP, SqlTypeName.VARCHAR, SqlTypeName.VARCHAR), - ImmutableList.of(ColumnType.LONG, ColumnType.STRING, ColumnType.STRING), + ImmutableList.of( + SqlTypeName.TIMESTAMP, + SqlTypeName.VARCHAR, + SqlTypeName.VARCHAR + ), + ImmutableList.of( + ColumnType.LONG, + ColumnType.STRING, + ColumnType.STRING + ), null ); @@ -206,7 +227,15 @@ public class SqlStatementResourceTest extends MSQTestBase private static final MSQTaskReport MSQ_SELECT_TASK_REPORT = new MSQTaskReport( FINISHED_SELECT_MSQ_QUERY, new MSQTaskReportPayload( - new MSQStatusReport(TaskState.SUCCESS, null, new ArrayDeque<>(), null, 0, 1, 2), + new MSQStatusReport( + TaskState.SUCCESS, + null, + new ArrayDeque<>(), + null, + 0, + 1, + 2 + ), MSQStagesReport.create( MSQTaskReportTest.QUERY_DEFINITION, ImmutableMap.of(), @@ -217,12 +246,27 @@ public class SqlStatementResourceTest extends MSQTestBase new CounterSnapshotsTree(), new MSQResultsReport( ImmutableList.of( - new MSQResultsReport.ColumnAndType("_time", ColumnType.LONG), - new MSQResultsReport.ColumnAndType("alias", ColumnType.STRING), - new MSQResultsReport.ColumnAndType("market", ColumnType.STRING) + new MSQResultsReport.ColumnAndType( + "_time", + ColumnType.LONG + ), + new MSQResultsReport.ColumnAndType( + "alias", + ColumnType.STRING + ), + new MSQResultsReport.ColumnAndType( + "market", + ColumnType.STRING + ) ), - ImmutableList.of(SqlTypeName.TIMESTAMP, SqlTypeName.VARCHAR, SqlTypeName.VARCHAR), - Yielders.each(Sequences.simple(RESULT_ROWS)), + ImmutableList.of( + SqlTypeName.TIMESTAMP, + SqlTypeName.VARCHAR, + SqlTypeName.VARCHAR + ), + Yielders.each( + Sequences.simple( + RESULT_ROWS)), null ) ) @@ -231,7 +275,15 @@ public class SqlStatementResourceTest extends MSQTestBase private static final MSQTaskReport MSQ_INSERT_TASK_REPORT = new MSQTaskReport( FINISHED_INSERT_MSQ_QUERY, new MSQTaskReportPayload( - new MSQStatusReport(TaskState.SUCCESS, null, new ArrayDeque<>(), null, 0, 1, 2), + new MSQStatusReport( + TaskState.SUCCESS, + null, + new ArrayDeque<>(), + null, + 0, + 1, + 2 + ), MSQStagesReport.create( MSQTaskReportTest.QUERY_DEFINITION, ImmutableMap.of(), @@ -254,10 +306,22 @@ public class SqlStatementResourceTest extends MSQTestBase private static final Map ROW1 = ImmutableMap.of("_time", 123, "alias", "foo", "market", "bar"); private static final Map ROW2 = ImmutableMap.of("_time", 234, "alias", "foo1", "market", "bar1"); - private static final ImmutableList COL_NAME_AND_TYPES = ImmutableList.of( - new ColNameAndType("_time", SqlTypeName.TIMESTAMP.getName(), ValueType.LONG.name()), - new ColNameAndType("alias", SqlTypeName.VARCHAR.getName(), ValueType.STRING.name()), - new ColNameAndType("market", SqlTypeName.VARCHAR.getName(), ValueType.STRING.name()) + private static final ImmutableList COL_NAME_AND_TYPES = ImmutableList.of( + new ColumnNameAndTypes( + "_time", + SqlTypeName.TIMESTAMP.getName(), + ValueType.LONG.name() + ), + new ColumnNameAndTypes( + "alias", + SqlTypeName.VARCHAR.getName(), + ValueType.STRING.name() + ), + new ColumnNameAndTypes( + "market", + SqlTypeName.VARCHAR.getName(), + ValueType.STRING.name() + ) ); private static final String FAILURE_MSG = "failure msg"; @@ -280,22 +344,19 @@ private static void setupMocks(OverlordClient indexingServiceClient) throws Json { Mockito.when(indexingServiceClient.taskStatus(ArgumentMatchers.eq(ACCEPTED_SELECT_MSQ_QUERY))) - .thenReturn(Futures.immediateFuture(new TaskStatusResponse( + .thenReturn(Futures.immediateFuture(new TaskStatusResponse(ACCEPTED_SELECT_MSQ_QUERY, new TaskStatusPlus( ACCEPTED_SELECT_MSQ_QUERY, - new TaskStatusPlus( - ACCEPTED_SELECT_MSQ_QUERY, - null, - MSQControllerTask.TYPE, - CREATED_TIME, - QUEUE_INSERTION_TIME, - null, - null, - null, - TaskLocation.unknown(), - null, - null - ) - ))); + null, + MSQControllerTask.TYPE, + CREATED_TIME, + QUEUE_INSERTION_TIME, + null, + null, + null, + TaskLocation.unknown(), + null, + null + )))); Mockito.when(indexingServiceClient.taskPayload(ArgumentMatchers.eq(ACCEPTED_SELECT_MSQ_QUERY))) .thenReturn(Futures.immediateFuture(new TaskPayloadResponse( @@ -304,22 +365,19 @@ private static void setupMocks(OverlordClient indexingServiceClient) throws Json ))); Mockito.when(indexingServiceClient.taskStatus(ArgumentMatchers.eq(RUNNING_SELECT_MSQ_QUERY))) - .thenReturn(Futures.immediateFuture(new TaskStatusResponse( + .thenReturn(Futures.immediateFuture(new TaskStatusResponse(RUNNING_SELECT_MSQ_QUERY, new TaskStatusPlus( RUNNING_SELECT_MSQ_QUERY, - new TaskStatusPlus( - RUNNING_SELECT_MSQ_QUERY, - null, - MSQControllerTask.TYPE, - CREATED_TIME, - QUEUE_INSERTION_TIME, - TaskState.RUNNING, - null, - null, - new TaskLocation("test", 0, 0), - null, - null - ) - ))); + null, + MSQControllerTask.TYPE, + CREATED_TIME, + QUEUE_INSERTION_TIME, + TaskState.RUNNING, + null, + null, + new TaskLocation("test", 0, 0), + null, + null + )))); Mockito.when(indexingServiceClient.taskPayload(RUNNING_SELECT_MSQ_QUERY)) .thenReturn(Futures.immediateFuture(new TaskPayloadResponse( @@ -329,22 +387,19 @@ private static void setupMocks(OverlordClient indexingServiceClient) throws Json Mockito.when(indexingServiceClient.taskStatus(ArgumentMatchers.eq(FINISHED_SELECT_MSQ_QUERY))) - .thenReturn(Futures.immediateFuture(new TaskStatusResponse( + .thenReturn(Futures.immediateFuture(new TaskStatusResponse(FINISHED_SELECT_MSQ_QUERY, new TaskStatusPlus( FINISHED_SELECT_MSQ_QUERY, - new TaskStatusPlus( - FINISHED_SELECT_MSQ_QUERY, - null, - MSQControllerTask.TYPE, - CREATED_TIME, - QUEUE_INSERTION_TIME, - TaskState.SUCCESS, - null, - 100L, - new TaskLocation("test", 0, 0), - null, - null - ) - ))); + null, + MSQControllerTask.TYPE, + CREATED_TIME, + QUEUE_INSERTION_TIME, + TaskState.SUCCESS, + null, + 100L, + new TaskLocation("test", 0, 0), + null, + null + )))); Mockito.when(indexingServiceClient.taskPayload(FINISHED_SELECT_MSQ_QUERY)) .thenReturn(Futures.immediateFuture(new TaskPayloadResponse( @@ -359,7 +414,8 @@ private static void setupMocks(OverlordClient indexingServiceClient) throws Json Mockito.when(indexingServiceClient.taskReportAsMap(FINISHED_SELECT_MSQ_QUERY)) .thenReturn(Futures.immediateFuture(mapper.readValue( - mapper.writeValueAsString(TaskReport.buildTaskReports(MSQ_SELECT_TASK_REPORT)), + mapper.writeValueAsString(TaskReport.buildTaskReports( + MSQ_SELECT_TASK_REPORT)), new TypeReference>() { } @@ -367,22 +423,19 @@ private static void setupMocks(OverlordClient indexingServiceClient) throws Json Mockito.when(indexingServiceClient.taskStatus(ArgumentMatchers.eq(ERRORED_SELECT_MSQ_QUERY))) - .thenReturn(Futures.immediateFuture(new TaskStatusResponse( + .thenReturn(Futures.immediateFuture(new TaskStatusResponse(ERRORED_SELECT_MSQ_QUERY, new TaskStatusPlus( ERRORED_SELECT_MSQ_QUERY, - new TaskStatusPlus( - ERRORED_SELECT_MSQ_QUERY, - null, - MSQControllerTask.TYPE, - CREATED_TIME, - QUEUE_INSERTION_TIME, - TaskState.FAILED, - null, - -1L, - TaskLocation.unknown(), - null, - FAILURE_MSG - ) - ))); + null, + MSQControllerTask.TYPE, + CREATED_TIME, + QUEUE_INSERTION_TIME, + TaskState.FAILED, + null, + -1L, + TaskLocation.unknown(), + null, + FAILURE_MSG + )))); Mockito.when(indexingServiceClient.taskPayload(ArgumentMatchers.eq(ERRORED_SELECT_MSQ_QUERY))) .thenReturn(Futures.immediateFuture(new TaskPayloadResponse( @@ -394,79 +447,67 @@ private static void setupMocks(OverlordClient indexingServiceClient) throws Json .thenReturn(Futures.immediateFuture(null)); Mockito.when(indexingServiceClient.taskStatus(ArgumentMatchers.eq(RUNNING_NON_MSQ_TASK))) - .thenReturn(Futures.immediateFuture(new TaskStatusResponse( + .thenReturn(Futures.immediateFuture(new TaskStatusResponse(RUNNING_NON_MSQ_TASK, new TaskStatusPlus( RUNNING_NON_MSQ_TASK, - new TaskStatusPlus( - RUNNING_NON_MSQ_TASK, - null, - null, - CREATED_TIME, - QUEUE_INSERTION_TIME, - TaskState.RUNNING, - null, - -1L, - TaskLocation.unknown(), - null, - null - ) - ))); + null, + null, + CREATED_TIME, + QUEUE_INSERTION_TIME, + TaskState.RUNNING, + null, + -1L, + TaskLocation.unknown(), + null, + null + )))); Mockito.when(indexingServiceClient.taskStatus(ArgumentMatchers.eq(FAILED_NON_MSQ_TASK))) - .thenReturn(Futures.immediateFuture(new TaskStatusResponse( + .thenReturn(Futures.immediateFuture(new TaskStatusResponse(FAILED_NON_MSQ_TASK, new TaskStatusPlus( FAILED_NON_MSQ_TASK, - new TaskStatusPlus( - FAILED_NON_MSQ_TASK, - null, - null, - CREATED_TIME, - QUEUE_INSERTION_TIME, - TaskState.FAILED, - null, - -1L, - TaskLocation.unknown(), - null, - FAILURE_MSG - ) - ))); + null, + null, + CREATED_TIME, + QUEUE_INSERTION_TIME, + TaskState.FAILED, + null, + -1L, + TaskLocation.unknown(), + null, + FAILURE_MSG + )))); Mockito.when(indexingServiceClient.taskStatus(ArgumentMatchers.eq(FINISHED_NON_MSQ_TASK))) - .thenReturn(Futures.immediateFuture(new TaskStatusResponse( + .thenReturn(Futures.immediateFuture(new TaskStatusResponse(FINISHED_NON_MSQ_TASK, new TaskStatusPlus( FINISHED_NON_MSQ_TASK, - new TaskStatusPlus( - FINISHED_NON_MSQ_TASK, - null, - IndexTask.TYPE, - CREATED_TIME, - QUEUE_INSERTION_TIME, - TaskState.SUCCESS, - null, - -1L, - TaskLocation.unknown(), - null, - null - ) - ))); + null, + IndexTask.TYPE, + CREATED_TIME, + QUEUE_INSERTION_TIME, + TaskState.SUCCESS, + null, + -1L, + TaskLocation.unknown(), + null, + null + )))); Mockito.when(indexingServiceClient.taskStatus(ArgumentMatchers.eq(ACCEPTED_INSERT_MSQ_TASK))) - .thenReturn(Futures.immediateFuture(new TaskStatusResponse( + .thenReturn(Futures.immediateFuture(new TaskStatusResponse(ACCEPTED_SELECT_MSQ_QUERY, new TaskStatusPlus( ACCEPTED_SELECT_MSQ_QUERY, - new TaskStatusPlus( - ACCEPTED_SELECT_MSQ_QUERY, - null, - MSQControllerTask.TYPE, - CREATED_TIME, - QUEUE_INSERTION_TIME, - null, - null, - null, - TaskLocation.unknown(), - null, - null - ) - ))); + null, + MSQControllerTask.TYPE, + CREATED_TIME, + QUEUE_INSERTION_TIME, + null, + null, + null, + TaskLocation.unknown(), + null, + null + )))); Mockito.when(indexingServiceClient.taskPayload(ArgumentMatchers.eq(ACCEPTED_INSERT_MSQ_TASK))) .thenReturn(Futures.immediateFuture(new TaskPayloadResponse( @@ -475,22 +516,19 @@ private static void setupMocks(OverlordClient indexingServiceClient) throws Json ))); Mockito.when(indexingServiceClient.taskStatus(ArgumentMatchers.eq(RUNNING_INSERT_MSQ_QUERY))) - .thenReturn(Futures.immediateFuture(new TaskStatusResponse( + .thenReturn(Futures.immediateFuture(new TaskStatusResponse(RUNNING_INSERT_MSQ_QUERY, new TaskStatusPlus( RUNNING_INSERT_MSQ_QUERY, - new TaskStatusPlus( - RUNNING_INSERT_MSQ_QUERY, - null, - MSQControllerTask.TYPE, - CREATED_TIME, - QUEUE_INSERTION_TIME, - TaskState.RUNNING, - null, - null, - new TaskLocation("test", 0, 0), - null, - null - ) - ))); + null, + MSQControllerTask.TYPE, + CREATED_TIME, + QUEUE_INSERTION_TIME, + TaskState.RUNNING, + null, + null, + new TaskLocation("test", 0, 0), + null, + null + )))); Mockito.when(indexingServiceClient.taskPayload(RUNNING_INSERT_MSQ_QUERY)) .thenReturn(Futures.immediateFuture(new TaskPayloadResponse( @@ -500,26 +538,24 @@ private static void setupMocks(OverlordClient indexingServiceClient) throws Json Mockito.when(indexingServiceClient.taskStatus(ArgumentMatchers.eq(FINISHED_INSERT_MSQ_QUERY))) - .thenReturn(Futures.immediateFuture(new TaskStatusResponse( + .thenReturn(Futures.immediateFuture(new TaskStatusResponse(FINISHED_INSERT_MSQ_QUERY, new TaskStatusPlus( FINISHED_INSERT_MSQ_QUERY, - new TaskStatusPlus( - FINISHED_INSERT_MSQ_QUERY, - null, - MSQControllerTask.TYPE, - CREATED_TIME, - QUEUE_INSERTION_TIME, - TaskState.SUCCESS, - null, - 100L, - new TaskLocation("test", 0, 0), - null, - null - ) - ))); + null, + MSQControllerTask.TYPE, + CREATED_TIME, + QUEUE_INSERTION_TIME, + TaskState.SUCCESS, + null, + 100L, + new TaskLocation("test", 0, 0), + null, + null + )))); Mockito.when(indexingServiceClient.taskReportAsMap(ArgumentMatchers.eq(FINISHED_INSERT_MSQ_QUERY))) .thenReturn(Futures.immediateFuture(mapper.readValue( - mapper.writeValueAsString(TaskReport.buildTaskReports(MSQ_INSERT_TASK_REPORT)), + mapper.writeValueAsString(TaskReport.buildTaskReports( + MSQ_INSERT_TASK_REPORT)), new TypeReference>() { } @@ -532,22 +568,19 @@ private static void setupMocks(OverlordClient indexingServiceClient) throws Json ))); Mockito.when(indexingServiceClient.taskStatus(ArgumentMatchers.eq(ERRORED_INSERT_MSQ_QUERY))) - .thenReturn(Futures.immediateFuture(new TaskStatusResponse( + .thenReturn(Futures.immediateFuture(new TaskStatusResponse(ERRORED_INSERT_MSQ_QUERY, new TaskStatusPlus( ERRORED_INSERT_MSQ_QUERY, - new TaskStatusPlus( - ERRORED_INSERT_MSQ_QUERY, - null, - MSQControllerTask.TYPE, - CREATED_TIME, - QUEUE_INSERTION_TIME, - TaskState.FAILED, - null, - -1L, - TaskLocation.unknown(), - null, - FAILURE_MSG - ) - ))); + null, + MSQControllerTask.TYPE, + CREATED_TIME, + QUEUE_INSERTION_TIME, + TaskState.FAILED, + null, + -1L, + TaskLocation.unknown(), + null, + FAILURE_MSG + )))); Mockito.when(indexingServiceClient.taskPayload(ArgumentMatchers.eq(ERRORED_INSERT_MSQ_QUERY))) .thenReturn(Futures.immediateFuture(new TaskPayloadResponse( @@ -565,19 +598,25 @@ public void testMSQSelectAcceptedQuery() { Response response = resource.doGetStatus(ACCEPTED_SELECT_MSQ_QUERY, makeOkRequest()); Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); - Assert.assertEquals(new SqlStatementResult( - ACCEPTED_SELECT_MSQ_QUERY, - SqlStatementState.ACCEPTED, - CREATED_TIME, - COL_NAME_AND_TYPES, - null, - null, - null - ), response.getEntity()); + Assert.assertEquals( + new SqlStatementResult( + ACCEPTED_SELECT_MSQ_QUERY, + SqlStatementState.ACCEPTED, + CREATED_TIME, + COL_NAME_AND_TYPES, + null, + null, + null + ), + response.getEntity() + ); assertExceptionMessage( - resource.doGetResults(ACCEPTED_SELECT_MSQ_QUERY, null, null, null, null, makeOkRequest()), - StringUtils.format("Query is [%s]. Please wait for it to complete.", SqlStatementState.ACCEPTED), + resource.doGetResults(ACCEPTED_SELECT_MSQ_QUERY, null, null, makeOkRequest()), + StringUtils.format( + "Query is [%s]. Please wait for it to complete.", + SqlStatementState.ACCEPTED + ), Response.Status.NOT_FOUND ); Assert.assertEquals( @@ -593,19 +632,25 @@ public void testMSQSelectRunningQuery() Response response = resource.doGetStatus(RUNNING_SELECT_MSQ_QUERY, makeOkRequest()); Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); - Assert.assertEquals(new SqlStatementResult( - RUNNING_SELECT_MSQ_QUERY, - SqlStatementState.RUNNING, - CREATED_TIME, - COL_NAME_AND_TYPES, - null, - null, - null - ), response.getEntity()); + Assert.assertEquals( + new SqlStatementResult( + RUNNING_SELECT_MSQ_QUERY, + SqlStatementState.RUNNING, + CREATED_TIME, + COL_NAME_AND_TYPES, + null, + null, + null + ), + response.getEntity() + ); assertExceptionMessage( - resource.doGetResults(RUNNING_SELECT_MSQ_QUERY, null, null, null, null, makeOkRequest()), - StringUtils.format("Query is [%s]. Please wait for it to complete.", SqlStatementState.RUNNING), + resource.doGetResults(RUNNING_SELECT_MSQ_QUERY, null, null, makeOkRequest()), + StringUtils.format( + "Query is [%s]. Please wait for it to complete.", + SqlStatementState.RUNNING + ), Response.Status.NOT_FOUND ); Assert.assertEquals( @@ -631,21 +676,15 @@ public void testFinishedSelectMSQQuery() throws Exception null, null, MSQControllerTask.DUMMY_DATASOURCE_FOR_SELECT, - RESULT_ROWS.stream().map(Arrays::asList).collect(Collectors.toList()) + RESULT_ROWS.stream() + .map(Arrays::asList) + .collect(Collectors.toList()) ), null ), response.getEntity()); - Response resultsResponse = resource.doGetResults( - FINISHED_SELECT_MSQ_QUERY, - null, - null, - null, - null, - makeOkRequest() - ); - Assert.assertEquals( - Response.Status.OK.getStatusCode(), resultsResponse.getStatus()); + Response resultsResponse = resource.doGetResults(FINISHED_SELECT_MSQ_QUERY, null, null, makeOkRequest()); + Assert.assertEquals(Response.Status.OK.getStatusCode(), resultsResponse.getStatus()); List> rows = new ArrayList<>(); rows.add(ROW1); @@ -664,8 +703,6 @@ public void testFinishedSelectMSQQuery() throws Exception FINISHED_SELECT_MSQ_QUERY, 1L, null, - null, - null, makeOkRequest() )) ); @@ -675,8 +712,6 @@ public void testFinishedSelectMSQQuery() throws Exception FINISHED_SELECT_MSQ_QUERY, 0L, 1L, - null, - null, makeOkRequest() )) ); @@ -686,21 +721,17 @@ public void testFinishedSelectMSQQuery() throws Exception FINISHED_SELECT_MSQ_QUERY, 0L, 3L, - null, - null, makeOkRequest() )) ); Assert.assertEquals( Response.Status.PRECONDITION_FAILED.getStatusCode(), - resource.doGetResults(FINISHED_SELECT_MSQ_QUERY, -1L, 3L, null, null, makeOkRequest()) - .getStatus() + resource.doGetResults(FINISHED_SELECT_MSQ_QUERY, -1L, 3L, makeOkRequest()).getStatus() ); Assert.assertEquals( Response.Status.PRECONDITION_FAILED.getStatusCode(), - resource.doGetResults(FINISHED_SELECT_MSQ_QUERY, null, -1L, null, null, makeOkRequest()) - .getStatus() + resource.doGetResults(FINISHED_SELECT_MSQ_QUERY, null, -1L, makeOkRequest()).getStatus() ); } @@ -710,13 +741,9 @@ public void testFinishedSelectMSQQuery() throws Exception public void testFailedMSQQuery() { for (String queryID : ImmutableList.of(ERRORED_SELECT_MSQ_QUERY, ERRORED_INSERT_MSQ_QUERY)) { + assertExceptionMessage(resource.doGetStatus(queryID, makeOkRequest()), FAILURE_MSG, Response.Status.OK); assertExceptionMessage( - resource.doGetStatus(queryID, makeOkRequest()), - FAILURE_MSG, - Response.Status.OK - ); - assertExceptionMessage( - resource.doGetResults(queryID, null, null, null, null, makeOkRequest()), + resource.doGetResults(queryID, null, null, makeOkRequest()), FAILURE_MSG, Response.Status.NOT_FOUND ); @@ -744,58 +771,36 @@ public void testFinishedInsertMSQQuery() throws Exception null ), response.getEntity()); - Response resultsResponse = resource.doGetResults( + Response resultsResponse = resource.doGetResults(FINISHED_INSERT_MSQ_QUERY, null, null, makeOkRequest()); + Assert.assertEquals(Response.Status.OK.getStatusCode(), resultsResponse.getStatus()); + + + Assert.assertNull(getResultRowsFromResponse(resource.doGetResults( FINISHED_INSERT_MSQ_QUERY, - null, - null, - null, + 1L, null, makeOkRequest() - ); - Assert.assertEquals( - Response.Status.OK.getStatusCode(), resultsResponse.getStatus()); - - - Assert.assertNull( - getResultRowsFromResponse(resource.doGetResults( - FINISHED_INSERT_MSQ_QUERY, - 1L, - null, - null, - null, - makeOkRequest() - )) - ); - Assert.assertNull( - getResultRowsFromResponse(resource.doGetResults( - FINISHED_INSERT_MSQ_QUERY, - 0L, - 1L, - null, - null, - makeOkRequest() - )) - ); - Assert.assertNull( - getResultRowsFromResponse(resource.doGetResults( - FINISHED_INSERT_MSQ_QUERY, - 0L, - 3L, - null, - null, - makeOkRequest() - )) - ); + ))); + Assert.assertNull(getResultRowsFromResponse(resource.doGetResults( + FINISHED_INSERT_MSQ_QUERY, + 0L, + 1L, + makeOkRequest() + ))); + Assert.assertNull(getResultRowsFromResponse(resource.doGetResults( + FINISHED_INSERT_MSQ_QUERY, + 0L, + 3L, + makeOkRequest() + ))); Assert.assertEquals( Response.Status.PRECONDITION_FAILED.getStatusCode(), - resource.doGetResults(FINISHED_INSERT_MSQ_QUERY, -1L, 3L, null, null, makeOkRequest()) - .getStatus() + resource.doGetResults(FINISHED_INSERT_MSQ_QUERY, -1L, 3L, makeOkRequest()).getStatus() ); Assert.assertEquals( Response.Status.PRECONDITION_FAILED.getStatusCode(), - resource.doGetResults(FINISHED_INSERT_MSQ_QUERY, null, -1L, null, null, makeOkRequest()) - .getStatus() + resource.doGetResults(FINISHED_INSERT_MSQ_QUERY, null, -1L, makeOkRequest()).getStatus() ); } @@ -804,18 +809,9 @@ public void testFinishedInsertMSQQuery() throws Exception public void testNonMSQTasks() { for (String queryID : ImmutableList.of(RUNNING_NON_MSQ_TASK, FAILED_NON_MSQ_TASK, FINISHED_NON_MSQ_TASK)) { - assertNullResponse( - resource.doGetStatus(queryID, makeOkRequest()), - Response.Status.NOT_FOUND - ); - assertNullResponse( - resource.doGetResults(queryID, null, null, null, null, makeOkRequest()), - Response.Status.NOT_FOUND - ); - assertNullResponse( - resource.deleteQuery(queryID, makeOkRequest()), - Response.Status.NOT_FOUND - ); + assertNullResponse(resource.doGetStatus(queryID, makeOkRequest()), Response.Status.NOT_FOUND); + assertNullResponse(resource.doGetResults(queryID, null, null, makeOkRequest()), Response.Status.NOT_FOUND); + assertNullResponse(resource.deleteQuery(queryID, makeOkRequest()), Response.Status.NOT_FOUND); } } @@ -824,19 +820,25 @@ public void testMSQInsertAcceptedQuery() { Response response = resource.doGetStatus(ACCEPTED_INSERT_MSQ_TASK, makeOkRequest()); Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); - Assert.assertEquals(new SqlStatementResult( - ACCEPTED_INSERT_MSQ_TASK, - SqlStatementState.ACCEPTED, - CREATED_TIME, - null, - null, - null, - null - ), response.getEntity()); + Assert.assertEquals( + new SqlStatementResult( + ACCEPTED_INSERT_MSQ_TASK, + SqlStatementState.ACCEPTED, + CREATED_TIME, + null, + null, + null, + null + ), + response.getEntity() + ); assertExceptionMessage( - resource.doGetResults(ACCEPTED_INSERT_MSQ_TASK, null, null, null, null, makeOkRequest()), - StringUtils.format("Query is [%s]. Please wait for it to complete.", SqlStatementState.ACCEPTED), + resource.doGetResults(ACCEPTED_INSERT_MSQ_TASK, null, null, makeOkRequest()), + StringUtils.format( + "Query is [%s]. Please wait for it to complete.", + SqlStatementState.ACCEPTED + ), Response.Status.NOT_FOUND ); Assert.assertEquals( @@ -850,19 +852,25 @@ public void testMSQInsertRunningQuery() { Response response = resource.doGetStatus(RUNNING_INSERT_MSQ_QUERY, makeOkRequest()); Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); - Assert.assertEquals(new SqlStatementResult( - RUNNING_INSERT_MSQ_QUERY, - SqlStatementState.RUNNING, - CREATED_TIME, - null, - null, - null, - null - ), response.getEntity()); + Assert.assertEquals( + new SqlStatementResult( + RUNNING_INSERT_MSQ_QUERY, + SqlStatementState.RUNNING, + CREATED_TIME, + null, + null, + null, + null + ), + response.getEntity() + ); assertExceptionMessage( - resource.doGetResults(RUNNING_INSERT_MSQ_QUERY, null, null, null, null, makeOkRequest()), - StringUtils.format("Query is [%s]. Please wait for it to complete.", SqlStatementState.RUNNING), + resource.doGetResults(RUNNING_INSERT_MSQ_QUERY, null, null, makeOkRequest()), + StringUtils.format( + "Query is [%s]. Please wait for it to complete.", + SqlStatementState.RUNNING + ), Response.Status.NOT_FOUND ); Assert.assertEquals( diff --git a/processing/src/main/java/org/apache/druid/error/DruidException.java b/processing/src/main/java/org/apache/druid/error/DruidException.java index 6acedf55fdb0..fb2d86b1e8b0 100644 --- a/processing/src/main/java/org/apache/druid/error/DruidException.java +++ b/processing/src/main/java/org/apache/druid/error/DruidException.java @@ -47,34 +47,33 @@ * no change should occur. * * Notes about exception messages - * + *

* Firstly, exception messages should always be written with the notions from the style conventions covered in * {@code dev/style-conventions.md}. Whenever possible, we should also try to provide an action to take to resolve * the issue. - * + *

* Secondly, given that the DruidException requires defining a target persona, exception messages should always be * written with that target persona in mind. Reviewers should use the targetPersona as added input to help validate * that an exception message in meaningful. - * + *

* For example, at the time that this exception was introduced, there is an exception that the router throws which is * an {@link org.apache.druid.java.util.common.ISE} with the message {@code "No default server found!"}. This * exception is thrown when the router is unable to find a broker to forward a request to. It is completely * meaningless to an end-user trying to run a query (what's a default server? why does it need to be found?). If we - * were to convert the exception to a DruidException and keep the same message, we should mark it as targetting the + * were to convert the exception to a DruidException and keep the same message, we should mark it as targeting the * DEVELOPER persona as that is the only persona who should actually be able to figure out what a default server is - * and why it is important. That said, does it makes sense for an exception that means "router cannot find a broker - * to forward the query to" to only be targetting the DEVELOPER? The answer to that is no, it's something that should - * really be made meaningful to a wider group. Some options could be - * - * USER persona: Cannot find a queryable server, contact your cluster administrator to validate that all services are - * operational - * - * OPERATOR persona: Router unable to find a broker, check that brokers are up and active - * + * and why it is important. That said, does it make sense for an exception that means "router cannot find a broker + * to forward the query to" to only be targeting the DEVELOPER? The answer to that is no, it's something that should + * really be made meaningful to a wider group. Some options could be + *

    + *
  • USER persona: Cannot find a queryable server, contact your cluster administrator to validate that all services are + * operational
  • + *
  • OPERATOR persona: Router unable to find a broker, check that brokers are up and active
  • + *
* The user-facing message doesn't talk about any Druid-specific concepts and just tries to relay a high-level * understanding of what happened. The admin-facing message includes Druid notions in it as it expects that an Admin * will understand the various node types of Druid. - * + *

* If we think about this error more, we will realize that it's fundamentally something wrong with the cluster setup, * which is something that we would expect an operator to be in charge of. So, we would pick the OPERATOR persona * message, which also allows us to include more specific information about what server was not found and provide a @@ -95,8 +94,8 @@ *

    *
  1. It identifies why the developer is creating the exception and who they believe can take action on it. * This context allows for code reviewers and other developers to evaluate the message with the persona in mind
  2. - *
  3. It can be used as a way to control which error messages should be routed where. For example, a user-targetted - * error message should be able to be exposed directly to the user, while an operator-targetted error message should + *
  4. It can be used as a way to control which error messages should be routed where. For example, a user-targeted + * error message should be able to be exposed directly to the user, while an operator-targeted error message should * perhaps be routed to the operators of the system instead of the end user firing a query.
  5. *
*

@@ -105,11 +104,11 @@ *

* The error code is a code that indicates a grouping of error messages. There is no forced structure around whether * a specific error code can be reused for different problems or not. That is, an error code like "general" will get - * reused in many different places as it's the basic error code used whenever a DruidException is created in-line. But, + * reused in many places as it's the basic error code used whenever a DruidException is created in-line. But, * we might decide that a specific type of error should be identified explicitly by its error code and should only mean * one thing, in which case that error code might only exist on a single error. *

- * The error message is a message written targetting the target persona. It should have values interpolated into it + * The error message is a message written targeting the target persona. It should have values interpolated into it * in order to be as meaningful as possible for the target persona without leaking potentially sensitive information. *

* The context is a place to add extra information about the error that is not necessarily interpolated into the @@ -132,7 +131,7 @@ public class DruidException extends RuntimeException { /** - * Starts building an "general" DruidException targetting the specific persona. + * Starts building a "general" DruidException targeting the specific persona. * * @param persona the target persona of the exception message * @return a builder that can be used to complete the creation of the DruidException @@ -266,7 +265,7 @@ public DruidException prependAndBuild(String msg, Object... args) } /** - * The persona that the message on a DruidException is targetting + * The persona that the message on a DruidException is targeting */ public enum Persona { @@ -307,7 +306,7 @@ public enum Category DEFENSIVE(500), /** * Means that the input provided was malformed in some way. Generally speaking, it is hoped that errors of this - * category have messages written either targetting the USER or ADMIN personas as those are the general users + * category have messages written either targeting the USER or ADMIN personas as those are the general users * of the APIs who could generate invalid inputs. */ INVALID_INPUT(400), diff --git a/sql/src/main/java/org/apache/druid/sql/DirectStatement.java b/sql/src/main/java/org/apache/druid/sql/DirectStatement.java index 0bcf0f684caf..21ebe9e17baa 100644 --- a/sql/src/main/java/org/apache/druid/sql/DirectStatement.java +++ b/sql/src/main/java/org/apache/druid/sql/DirectStatement.java @@ -230,8 +230,8 @@ public ResultSet plan() catch (RelOptPlanner.CannotPlanException e) { // Not sure if this is even thrown here. throw DruidException.forPersona(DruidException.Persona.DEVELOPER) - .ofCategory(DruidException.Category.UNCATEGORIZED) - .build(e, "Problem planning SQL query"); + .ofCategory(DruidException.Category.UNCATEGORIZED) + .build(e, "Problem planning SQL query"); } catch (RuntimeException e) { state = State.FAILED; From 7634124a3a0debc792249d179912ebd2b1c9a6f8 Mon Sep 17 00:00:00 2001 From: cryptoe Date: Mon, 26 Jun 2023 22:37:44 +0530 Subject: [PATCH 08/15] 1. Adding resource helper 2. Moving to DruidExceptions --- .../apache/druid/msq/sql/SqlTaskStatus.java | 1 - .../msq/sql/entity/SqlStatementResult.java | 16 +- .../sql/resources/SqlStatementResource.java | 530 +++++------------- .../msq/util/SqlStatementResourceHelper.java | 272 +++++++++ .../sql/SqlMsqStatementResourcePostTest.java | 3 - .../msq/sql/SqlStatementResourceTest.java | 47 +- 6 files changed, 450 insertions(+), 419 deletions(-) create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/SqlStatementResourceHelper.java diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlTaskStatus.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlTaskStatus.java index b1aff6772823..9c92a2589f2d 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlTaskStatus.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/SqlTaskStatus.java @@ -26,7 +26,6 @@ import org.apache.druid.error.ErrorResponse; import org.apache.druid.indexer.TaskState; import org.apache.druid.msq.sql.resources.SqlTaskResource; -import org.apache.druid.query.QueryException; import javax.annotation.Nullable; import java.util.Objects; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/SqlStatementResult.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/SqlStatementResult.java index 4d6ba7099658..967f4471becc 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/SqlStatementResult.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/SqlStatementResult.java @@ -43,7 +43,7 @@ public class SqlStatementResult private final List sqlRowSignature; @Nullable - private final Long durationInMs; + private final Long durationMs; @Nullable private final ResultSetInformation resultSetInformation; @@ -75,7 +75,7 @@ public SqlStatementResult( this.state = state; this.createdAt = createdAt; this.sqlRowSignature = sqlRowSignature; - this.durationInMs = durationMs; + this.durationMs = durationMs; this.resultSetInformation = resultSetInformation; this.queryException = queryException; } @@ -109,9 +109,9 @@ public List getSqlRowSignature() @JsonProperty @Nullable @JsonInclude(JsonInclude.Include.NON_NULL) - public Long getDurationInMs() + public Long getDurationMs() { - return durationInMs; + return durationMs; } @JsonProperty @@ -144,8 +144,8 @@ public boolean equals(Object o) createdAt, that.createdAt ) && Objects.equals(sqlRowSignature, that.sqlRowSignature) && Objects.equals( - durationInMs, - that.durationInMs + durationMs, + that.durationMs ) && Objects.equals(resultSetInformation, that.resultSetInformation) && Objects.equals( queryException, that.queryException @@ -155,7 +155,7 @@ public boolean equals(Object o) @Override public int hashCode() { - return Objects.hash(queryId, state, createdAt, sqlRowSignature, durationInMs, resultSetInformation, queryException); + return Objects.hash(queryId, state, createdAt, sqlRowSignature, durationMs, resultSetInformation, queryException); } @Override @@ -166,7 +166,7 @@ public String toString() ", state=" + state + ", createdAt=" + createdAt + ", sqlRowSignature=" + sqlRowSignature + - ", durationInMs=" + durationInMs + + ", durationInMs=" + durationMs + ", resultSetInformation=" + resultSetInformation + ", queryException=" + queryException + '}'; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java index f33ec03391a4..029d4fd31ed2 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java @@ -25,15 +25,13 @@ import com.google.common.io.CountingOutputStream; import com.google.common.util.concurrent.ListenableFuture; import com.google.inject.Inject; -import org.apache.calcite.plan.RelOptPlanner; -import org.apache.calcite.sql.type.SqlTypeName; import org.apache.druid.client.indexing.TaskPayloadResponse; import org.apache.druid.client.indexing.TaskStatusResponse; -import org.apache.druid.common.exception.SanitizableException; import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.ErrorResponse; +import org.apache.druid.error.QueryExceptionCompat; import org.apache.druid.guice.annotations.MSQ; -import org.apache.druid.indexer.TaskLocation; -import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Pair; @@ -44,26 +42,18 @@ import org.apache.druid.java.util.common.guava.Yielders; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.msq.indexing.MSQControllerTask; -import org.apache.druid.msq.indexing.TaskReportMSQDestination; import org.apache.druid.msq.sql.MSQTaskQueryMaker; import org.apache.druid.msq.sql.MSQTaskSqlEngine; import org.apache.druid.msq.sql.SqlStatementState; import org.apache.druid.msq.sql.entity.ColumnNameAndTypes; import org.apache.druid.msq.sql.entity.ResultSetInformation; import org.apache.druid.msq.sql.entity.SqlStatementResult; -import org.apache.druid.query.BadQueryException; +import org.apache.druid.msq.util.SqlStatementResourceHelper; import org.apache.druid.query.ExecutionMode; -import org.apache.druid.query.QueryCapacityExceededException; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryException; -import org.apache.druid.query.QueryInterruptedException; -import org.apache.druid.query.QueryTimeoutException; -import org.apache.druid.query.QueryUnsupportedException; -import org.apache.druid.query.ResourceLimitExceededException; import org.apache.druid.rpc.indexing.OverlordClient; -import org.apache.druid.segment.column.ColumnType; import org.apache.druid.server.QueryResponse; -import org.apache.druid.server.initialization.ServerConfig; import org.apache.druid.server.security.Access; import org.apache.druid.server.security.AuthenticationResult; import org.apache.druid.server.security.AuthorizationUtils; @@ -71,14 +61,11 @@ import org.apache.druid.server.security.ForbiddenException; import org.apache.druid.sql.DirectStatement; import org.apache.druid.sql.HttpStatement; -import org.apache.druid.sql.SqlPlanningException; import org.apache.druid.sql.SqlRowTransformer; import org.apache.druid.sql.SqlStatementFactory; -import org.apache.druid.sql.calcite.planner.ColumnMappings; import org.apache.druid.sql.http.ResultFormat; import org.apache.druid.sql.http.SqlQuery; import org.apache.druid.sql.http.SqlResource; -import org.apache.http.HttpStatus; import javax.servlet.http.HttpServletRequest; import javax.ws.rs.Consumes; @@ -94,9 +81,7 @@ import javax.ws.rs.core.Response; import javax.ws.rs.core.StreamingOutput; import java.io.IOException; -import java.util.ArrayList; import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; @@ -108,7 +93,6 @@ public class SqlStatementResource private static final Logger log = new Logger(SqlStatementResource.class); private final SqlStatementFactory msqSqlStatementFactory; - private final ServerConfig serverConfig; private final AuthorizerMapper authorizerMapper; private final ObjectMapper jsonMapper; private final OverlordClient overlordClient; @@ -117,14 +101,12 @@ public class SqlStatementResource @Inject public SqlStatementResource( final @MSQ SqlStatementFactory msqSqlStatementFactory, - final ServerConfig serverConfig, final AuthorizerMapper authorizerMapper, final ObjectMapper jsonMapper, final OverlordClient overlordClient ) { this.msqSqlStatementFactory = msqSqlStatementFactory; - this.serverConfig = serverConfig; this.authorizerMapper = authorizerMapper; this.jsonMapper = jsonMapper; this.overlordClient = overlordClient; @@ -166,20 +148,16 @@ public Response doPost(final SqlQuery sqlQuery, @Context final HttpServletReques ); if (ExecutionMode.ASYNC != executionMode) { return buildNonOkResponse( - HttpStatus.SC_UNPROCESSABLE_ENTITY, - new QueryException( - QueryException.UNSUPPORTED_OPERATION_ERROR_CODE, - StringUtils.format( - "The statement sql api only supports sync mode[%s]. Please set context parameter [%s=%s] in the payload", - ExecutionMode.ASYNC, - QueryContexts.CTX_EXECUTION_MODE, - ExecutionMode.ASYNC - ), - null, - null, - null - ), - stmt.sqlQueryId() + DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build( + StringUtils.format( + "The statement sql api only supports sync mode[%s]. Please set context parameter [%s=%s] in the context payload", + ExecutionMode.ASYNC, + QueryContexts.CTX_EXECUTION_MODE, + ExecutionMode.ASYNC + ) + ) ); } @@ -201,46 +179,32 @@ public Response doPost(final SqlQuery sqlQuery, @Context final HttpServletReques return buildStandardResponse(sequence, sqlQuery, sqlQueryId, rowTransformer); } } - // Kitchen-sinking the errors since they are all unchecked. - // Just copied from SqlResource. - catch (QueryCapacityExceededException cap) { - stmt.reporter().failed(cap); - return buildNonOkResponse(QueryCapacityExceededException.STATUS_CODE, cap, sqlQueryId); - } - catch (QueryUnsupportedException unsupported) { - stmt.reporter().failed(unsupported); - return buildNonOkResponse(QueryUnsupportedException.STATUS_CODE, unsupported, sqlQueryId); - } - catch (QueryTimeoutException timeout) { - stmt.reporter().failed(timeout); - return buildNonOkResponse(QueryTimeoutException.STATUS_CODE, timeout, sqlQueryId); - } - catch (SqlPlanningException | ResourceLimitExceededException e) { + catch (DruidException e) { stmt.reporter().failed(e); - return buildNonOkResponse(BadQueryException.STATUS_CODE, e, sqlQueryId); + return buildNonOkResponse(e); } - catch (ForbiddenException e) { - // No request logs for forbidden queries; same as SqlResource - throw (ForbiddenException) serverConfig.getErrorResponseTransformStrategy() - .transformIfNeeded(e); // let ForbiddenExceptionMapper handle this + catch (QueryException queryException) { + stmt.reporter().failed(queryException); + final DruidException underlyingException = DruidException.fromFailure(new QueryExceptionCompat(queryException)); + return buildNonOkResponse(underlyingException); } - catch (RelOptPlanner.CannotPlanException e) { - stmt.reporter().failed(e); - SqlPlanningException spe = new SqlPlanningException( - SqlPlanningException.PlanningError.UNSUPPORTED_SQL_ERROR, - e.getMessage() + catch (ForbiddenException e) { + log.debug("Got forbidden request for reason [%s]", e.getErrorMessage()); + return buildNonOkResponse( + DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.FORBIDDEN) + .build(Access.DEFAULT_ERROR_MESSAGE) ); - return buildNonOkResponse(BadQueryException.STATUS_CODE, spe, sqlQueryId); } - // Calcite throws a java.lang.AssertionError which is type Error not Exception. Using Throwable catches both. - catch (Throwable e) { + // Calcite throws java.lang.AssertionError at various points in planning/validation. + catch (AssertionError | Exception e) { stmt.reporter().failed(e); log.noStackTrace().warn(e, "Failed to handle query: %s", sqlQueryId); return buildNonOkResponse( - Response.Status.INTERNAL_SERVER_ERROR.getStatusCode(), - QueryInterruptedException.wrapIfNeeded(e), - sqlQueryId + DruidException.forPersona(DruidException.Persona.DEVELOPER) + .ofCategory(DruidException.Category.UNCATEGORIZED) + .build(e.getMessage()) ); } finally { @@ -272,15 +236,22 @@ public Response doGetStatus( return Response.status(Response.Status.NOT_FOUND).build(); } } - catch (ForbiddenException e) { - throw (ForbiddenException) serverConfig.getErrorResponseTransformStrategy().transformIfNeeded(e); + catch (DruidException e) { + return buildNonOkResponse(e); } - catch (QueryException e) { - return buildNonOkResponse(Response.Status.INTERNAL_SERVER_ERROR.getStatusCode(), e, queryId); + catch (ForbiddenException e) { + log.debug("Got forbidden request for reason [%s]", e.getErrorMessage()); + return buildNonOkResponse( + DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.FORBIDDEN) + .build(Access.DEFAULT_ERROR_MESSAGE) + ); } catch (Exception e) { log.noStackTrace().warn(e, "Failed to handle query: %s", queryId); - throw e; + return buildNonOkResponse(DruidException.forPersona(DruidException.Persona.DEVELOPER) + .ofCategory(DruidException.Category.UNCATEGORIZED) + .build(e, "Failed to handle query: [%s]", queryId)); } } @@ -296,39 +267,31 @@ public Response doGetResults( { if (offset != null && offset < 0) { return buildNonOkResponse( - Response.Status.PRECONDITION_FAILED.getStatusCode(), - new QueryException( - null, - "offset cannot be negative. Please pass a positive number.", - null, - null, - null - ), - queryId + DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build( + "offset cannot be negative. Please pass a positive number." + ) ); } if (numberOfRows != null && numberOfRows < 0) { return buildNonOkResponse( - Response.Status.PRECONDITION_FAILED.getStatusCode(), - new QueryException( - null, - "numRows cannot be negative. Please pass a positive number.", - null, - null, - null - ), - queryId + DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build( + "numRows cannot be negative. Please pass a positive number." + ) ); } final long start = offset == null ? 0 : offset; - final long last = getLastIndex(numberOfRows, start); + final long last = SqlStatementResourceHelper.getLastIndex(numberOfRows, start); try { AuthorizationUtils.authorizeAllResourceActions(req, Collections.emptyList(), authorizerMapper); final AuthenticationResult authenticationResult = AuthorizationUtils.authenticationResultFromRequest(req); - TaskStatusResponse taskResponse = overlordWork(overlordClient.taskStatus(queryId)); + TaskStatusResponse taskResponse = contactOverlord(overlordClient.taskStatus(queryId)); if (taskResponse == null) { return Response.status(Response.Status.NOT_FOUND).build(); } @@ -337,32 +300,35 @@ public Response doGetResults( if (statusPlus == null || !MSQControllerTask.TYPE.equals(statusPlus.getType())) { return Response.status(Response.Status.NOT_FOUND).build(); } - SqlStatementState sqlStatementState = getSqlStatementState(statusPlus); + SqlStatementState sqlStatementState = SqlStatementResourceHelper.getSqlStatementState(statusPlus); if (sqlStatementState == SqlStatementState.RUNNING || sqlStatementState == SqlStatementState.ACCEPTED) { - return buildNonOkResponse(Response.Status.NOT_FOUND.getStatusCode(), new QueryException( - null, - StringUtils.format( - "Query is [%s]. Please wait for it to complete.", - sqlStatementState - ), - null, - null, - null - ), queryId); + return buildNonOkResponse( + DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build( + "Query[%s] is currently in [%s] state. Please wait for it to complete.", + queryId, + sqlStatementState + ) + ); } else if (sqlStatementState == SqlStatementState.FAILED) { return buildNonOkResponse( - Response.Status.NOT_FOUND.getStatusCode(), - new QueryException(null, statusPlus.getErrorMsg(), null, null, null), - queryId + DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build( + "Query[%s] failed. Hit status api for more details.", + queryId + ) ); } else { MSQControllerTask msqControllerTask = getMSQControllerTaskOrThrow(queryId, authenticationResult.getIdentity()); - Optional> signature = getSignature(msqControllerTask); + Optional> signature = SqlStatementResourceHelper.getSignature(msqControllerTask); if (!signature.isPresent()) { return Response.ok().build(); } - Optional> results = getResults(getPayload(overlordWork(overlordClient.taskReportAsMap(queryId)))); + Optional> results = SqlStatementResourceHelper.getResults(SqlStatementResourceHelper.getPayload( + contactOverlord(overlordClient.taskReportAsMap(queryId)))); return Response.ok((StreamingOutput) outputStream -> { CountingOutputStream os = new CountingOutputStream(outputStream); @@ -392,20 +358,27 @@ public Response doGetResults( } } - catch (ForbiddenException e) { - throw (ForbiddenException) serverConfig.getErrorResponseTransformStrategy().transformIfNeeded(e); + catch (DruidException e) { + return buildNonOkResponse(e); } - catch (QueryException e) { - return buildNonOkResponse(Response.Status.INTERNAL_SERVER_ERROR.getStatusCode(), e, queryId); + catch (ForbiddenException e) { + log.debug("Got forbidden request for reason [%s]", e.getErrorMessage()); + return buildNonOkResponse( + DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.FORBIDDEN) + .build(Access.DEFAULT_ERROR_MESSAGE) + ); } catch (Exception e) { log.noStackTrace().warn(e, "Failed to handle query: %s", queryId); - throw e; + return buildNonOkResponse(DruidException.forPersona(DruidException.Persona.DEVELOPER) + .ofCategory(DruidException.Category.UNCATEGORIZED) + .build(e, "Failed to handle query: [%s]", queryId)); } } /** - * Queries can be canceled while in anystate. Canceling a query that has already completed will remove its results. + * Queries can be canceled while in any {@link SqlStatementState}. Canceling a query that has already completed will be a no-op. * * @param queryId queryId * @param req httpServletRequest @@ -435,7 +408,7 @@ public Response deleteQuery(@PathParam("id") final String queryId, @Context fina return Response.status(Response.Status.ACCEPTED).build(); case SUCCESS: case FAILED: - // we would also want to clean up the + // we would also want to clean up the results in the future. return Response.ok().build(); default: throw new ISE("Illegal State[%s] encountered", sqlStatementResult.get().getState()); @@ -444,18 +417,23 @@ public Response deleteQuery(@PathParam("id") final String queryId, @Context fina } else { return Response.status(Response.Status.NOT_FOUND).build(); } - - } - catch (ForbiddenException e) { - throw (ForbiddenException) serverConfig.getErrorResponseTransformStrategy().transformIfNeeded(e); + catch (DruidException e) { + return buildNonOkResponse(e); } - catch (QueryException e) { - return buildNonOkResponse(Response.Status.INTERNAL_SERVER_ERROR.getStatusCode(), e, queryId); + catch (ForbiddenException e) { + log.debug("Got forbidden request for reason [%s]", e.getErrorMessage()); + return buildNonOkResponse( + DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.FORBIDDEN) + .build(Access.DEFAULT_ERROR_MESSAGE) + ); } catch (Exception e) { log.noStackTrace().warn(e, "Failed to handle query: %s", queryId); - throw e; + return buildNonOkResponse(DruidException.forPersona(DruidException.Persona.DEVELOPER) + .ofCategory(DruidException.Category.UNCATEGORIZED) + .build(e, "Failed to handle query: [%s]", queryId)); } } @@ -527,100 +505,29 @@ private Response buildTaskResponse(Sequence sequence, String user) throw new RE("Expected a single row but got [%d] rows. Please check broker logs for more information.", numRows); } String taskId = (String) rows.get(0)[0]; - try { - Optional statementResult = getStatementStatus(taskId, user, true); - - if (statementResult.isPresent()) { - return Response.status(Response.Status.OK).entity(statementResult.get()).build(); - } else { - return buildNonOkResponse( - Response.Status.INTERNAL_SERVER_ERROR.getStatusCode(), - new QueryException( - null, - StringUtils.format( - "Unable to find associated task for query id %s", - taskId - ), - null, - null, - null - ), - taskId - ); - } - } - catch (QueryException e) { - return buildNonOkResponse(Response.Status.OK.getStatusCode(), e, taskId); - } - } - - private Response buildNonOkResponse(int status, SanitizableException e, String sqlQueryId) - { - // Though transformIfNeeded returns an exception, its purpose is to return - // a QueryException - Exception cleaned = serverConfig.getErrorResponseTransformStrategy().transformIfNeeded(e); - return Response.status(status).entity(cleaned).build(); - } + Optional statementResult = getStatementStatus(taskId, user, true); - private static Optional> getSignature( - MSQControllerTask msqControllerTask - ) - { - // only populate signature for select q's - if (msqControllerTask.getQuerySpec().getDestination().getClass() == TaskReportMSQDestination.class) { - ColumnMappings columnMappings = msqControllerTask.getQuerySpec().getColumnMappings(); - List sqlTypeNames = msqControllerTask.getSqlTypeNames(); - if (sqlTypeNames == null || sqlTypeNames.size() != columnMappings.size()) { - return Optional.empty(); - } - List nativeTypeNames = msqControllerTask.getNativeTypeNames(); - if (nativeTypeNames == null || nativeTypeNames.size() != columnMappings.size()) { - return Optional.empty(); - } - List signature = new ArrayList<>(columnMappings.size()); - int index = 0; - for (String colName : columnMappings.getOutputColumnNames()) { - signature.add(new ColumnNameAndTypes( - colName, - sqlTypeNames.get(index).getName(), - nativeTypeNames.get(index).asTypeString() - )); - index++; - } - return Optional.of(signature); + if (statementResult.isPresent()) { + return Response.status(Response.Status.OK).entity(statementResult.get()).build(); + } else { + return buildNonOkResponse( + DruidException.forPersona(DruidException.Persona.DEVELOPER) + .ofCategory(DruidException.Category.DEFENSIVE).build( + "Unable to find associated task for query id [%s]. Contact cluster admin to check overlord logs for [%s]", + taskId, + taskId + ) + ); } - return Optional.empty(); } - - private void checkTaskPayloadOrThrow(TaskPayloadResponse taskPayloadResponse, String queryId) throws QueryException + private Response buildNonOkResponse(DruidException exception) { - if (taskPayloadResponse == null || taskPayloadResponse.getPayload() == null) { - throw new QueryException( - QueryException.UNKNOWN_EXCEPTION_ERROR_CODE, - StringUtils.format( - "Could not get payload details for query[%s] from the overlord", - queryId - ), - null, - null, - null - ); - } - - if (MSQControllerTask.class != taskPayloadResponse.getPayload().getClass()) { - throw new QueryException( - QueryException.UNKNOWN_EXCEPTION_ERROR_CODE, - StringUtils.format( - "Fetched an unexpected payload for query[%s] from the overlord.", - queryId - ), - null, - null, - null - ); - } + return Response + .status(exception.getStatusCode()) + .entity(new ErrorResponse(exception)) + .build(); } private Optional getSampleResults( @@ -631,8 +538,12 @@ private Optional getSampleResults( ) { if (sqlStatementState == SqlStatementState.SUCCESS) { - Map payload = getPayload(overlordWork(overlordClient.taskReportAsMap(asyncResultId))); - Optional> rowsAndSize = getRowsAndSizeFromPayload(payload, isSelectQuery); + Map payload = SqlStatementResourceHelper.getPayload(contactOverlord(overlordClient.taskReportAsMap( + asyncResultId))); + Optional> rowsAndSize = SqlStatementResourceHelper.getRowsAndSizeFromPayload( + payload, + isSelectQuery + ); return Optional.of(new ResultSetInformation( null, // since the rows can be sampled, get the number of rows from counters @@ -640,60 +551,18 @@ private Optional getSampleResults( rowsAndSize.orElse(new Pair<>(null, null)).rhs, dataSource, // only populate sample results in case a select query is successful - isSelectQuery ? getResults(payload).orElse(null) : null + isSelectQuery ? SqlStatementResourceHelper.getResults(payload).orElse(null) : null )); } else { return Optional.empty(); } } - private Optional> getRowsAndSizeFromPayload(Map payload, boolean isSelectQuery) - { - List stages = getList(payload, "stages"); - if (stages == null || stages.isEmpty()) { - return Optional.empty(); - } else { - int maxStage = stages.size() - 1; // Last stage output is the total number of rows returned to the end user. - Map counterMap = getMap(getMap(payload, "counters"), String.valueOf(maxStage)); - long rows = -1L; - long sizeInBytes = -1L; - if (counterMap == null) { - return Optional.empty(); - } - for (Map.Entry worker : counterMap.entrySet()) { - Object workerChannels = worker.getValue(); - if (workerChannels == null || !(workerChannels instanceof Map)) { - return Optional.empty(); - } - if (isSelectQuery) { - Object output = ((Map) workerChannels).get("output"); - if (output != null && output instanceof Map) { - List rowsPerChannel = (List) ((Map) output).get("rows"); - List bytesPerChannel = (List) ((Map) output).get("bytes"); - for (Integer row : rowsPerChannel) { - rows = rows + row; - } - for (Integer bytes : bytesPerChannel) { - sizeInBytes = sizeInBytes + bytes; - } - } - } else { - Object output = ((Map) workerChannels).get("segmentGenerationProgress"); - if (output != null && output instanceof Map) { - rows += (Integer) ((Map) output).get("rowsPushed"); - } - } - } - - return Optional.of(new Pair<>(rows == -1L ? null : rows + 1, sizeInBytes == -1L ? null : sizeInBytes + 1)); - } - } - private Optional getStatementStatus(String queryId, String currentUser, boolean withResults) - throws QueryException, ForbiddenException + throws DruidException { - TaskStatusResponse taskResponse = overlordWork(overlordClient.taskStatus(queryId)); + TaskStatusResponse taskResponse = contactOverlord(overlordClient.taskStatus(queryId)); if (taskResponse == null) { return Optional.empty(); } @@ -705,43 +574,19 @@ private Optional getStatementStatus(String queryId, String c // since we need the controller payload for auth checks. MSQControllerTask msqControllerTask = getMSQControllerTaskOrThrow(queryId, currentUser); - SqlStatementState sqlStatementState = getSqlStatementState(statusPlus); + SqlStatementState sqlStatementState = SqlStatementResourceHelper.getSqlStatementState(statusPlus); if (SqlStatementState.FAILED == sqlStatementState) { - Map exceptionDetails = getQueryExceptionDetails(getPayload(overlordWork(overlordClient.taskReportAsMap( - queryId)))); - Map exception = getMap(exceptionDetails, "error"); - if (exceptionDetails == null || exception == null) { - return Optional.of(new SqlStatementResult( - queryId, - sqlStatementState, - taskResponse.getStatus().getCreatedTime(), - null, - taskResponse.getStatus().getDuration(), - null, - new QueryException(null, statusPlus.getErrorMsg(), null, null, null) - )); - } - - final String errorMessage = String.valueOf(exception.getOrDefault("errorMessage", statusPlus.getErrorMsg())); - exception.remove("errorMessage"); - String host = (String) exceptionDetails.getOrDefault("host", null); - Map stringException = new HashMap<>(); - for (Map.Entry exceptionKeys : exception.entrySet()) { - stringException.put(exceptionKeys.getKey(), String.valueOf(exceptionKeys.getValue())); - } - return Optional.of(new SqlStatementResult( + return SqlStatementResourceHelper.getExceptionPayload( queryId, + taskResponse, + statusPlus, sqlStatementState, - taskResponse.getStatus().getCreatedTime(), - null, - taskResponse.getStatus().getDuration(), - null, - new QueryException(null, errorMessage, null, host, stringException) - )); + contactOverlord(overlordClient.taskReportAsMap( + queryId)) + ); } else { - Optional> signature = getSignature(msqControllerTask); - + Optional> signature = SqlStatementResourceHelper.getSignature(msqControllerTask); return Optional.of(new SqlStatementResult( queryId, sqlStatementState, @@ -759,120 +604,33 @@ private Optional getStatementStatus(String queryId, String c } } - private Map getQueryExceptionDetails(Map payload) - { - return getMap(getMap(payload, "status"), "errorReport"); - } - private MSQControllerTask getMSQControllerTaskOrThrow(String queryId, String currentUser) throws ForbiddenException + private MSQControllerTask getMSQControllerTaskOrThrow(String queryId, String currentUser) { - TaskPayloadResponse taskPayloadResponse = overlordWork(overlordClient.taskPayload(queryId)); - checkTaskPayloadOrThrow(taskPayloadResponse, queryId); + TaskPayloadResponse taskPayloadResponse = contactOverlord(overlordClient.taskPayload(queryId)); + SqlStatementResourceHelper.isMSQPayload(taskPayloadResponse, queryId); MSQControllerTask msqControllerTask = (MSQControllerTask) taskPayloadResponse.getPayload(); if (currentUser == null || !currentUser.equals(msqControllerTask.getQuerySpec() .getQuery() .getContext() .get(MSQTaskQueryMaker.USER_KEY))) { - throw new ForbiddenException(); + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.FORBIDDEN) + .build(Access.DEFAULT_ERROR_MESSAGE); } return msqControllerTask; } - private T overlordWork(final ListenableFuture future) + private T contactOverlord(final ListenableFuture future) { try { return FutureUtils.getUnchecked(future, true); } catch (RuntimeException e) { - throw new QueryException(null, "Unable to contact overlord " + e.getMessage(), null, null, null); + throw DruidException.forPersona(DruidException.Persona.DEVELOPER) + .ofCategory(DruidException.Category.UNCATEGORIZED) + .build("Unable to contact overlord " + e.getMessage()); } } - - private static SqlStatementState getSqlStatementState(TaskStatusPlus taskStatusPlus) - { - TaskState state = taskStatusPlus.getStatusCode(); - if (state == null) { - return SqlStatementState.ACCEPTED; - } - - switch (state) { - case FAILED: - return SqlStatementState.FAILED; - case RUNNING: - if (TaskLocation.unknown().equals(taskStatusPlus.getLocation())) { - return SqlStatementState.ACCEPTED; - } else { - return SqlStatementState.RUNNING; - } - case SUCCESS: - return SqlStatementState.SUCCESS; - default: - throw new ISE("Unrecognized state[%s] found.", state); - } - } - - @SuppressWarnings("unchecked") - private Map getMap(Map map, String key) - { - if (map == null) { - return null; - } - return (Map) map.get(key); - } - - @SuppressWarnings("rawtypes") - private List getList(Map map, String key) - { - if (map == null) { - return null; - } - return (List) map.get(key); - } - - /** - * Get results from report - */ - @SuppressWarnings("unchecked") - private Optional> getResults(Map payload) - { - Map resultsHolder = getMap(payload, "results"); - - if (resultsHolder == null) { - return Optional.empty(); - } - - List data = (List) resultsHolder.get("results"); - List rows = new ArrayList<>(); - if (data != null) { - rows.addAll(data); - } - return Optional.of(rows); - } - - private Map getPayload(Map results) - { - Map msqReport = getMap(results, "multiStageQuery"); - Map payload = getMap(msqReport, "payload"); - return payload; - } - - private static long getLastIndex(Long numberOfRows, long start) - { - final long last; - if (numberOfRows == null) { - last = Long.MAX_VALUE; - } else { - long finalIndex; - try { - finalIndex = Math.addExact(start, numberOfRows); - } - catch (ArithmeticException e) { - finalIndex = Long.MAX_VALUE; - } - last = finalIndex; - } - return last; - } - } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/SqlStatementResourceHelper.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/SqlStatementResourceHelper.java new file mode 100644 index 000000000000..ba15b927ca33 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/SqlStatementResourceHelper.java @@ -0,0 +1,272 @@ +/* + * 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.msq.util; + +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.druid.client.indexing.TaskPayloadResponse; +import org.apache.druid.client.indexing.TaskStatusResponse; +import org.apache.druid.error.DruidException; +import org.apache.druid.indexer.TaskLocation; +import org.apache.druid.indexer.TaskState; +import org.apache.druid.indexer.TaskStatusPlus; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.msq.indexing.MSQControllerTask; +import org.apache.druid.msq.indexing.TaskReportMSQDestination; +import org.apache.druid.msq.sql.SqlStatementState; +import org.apache.druid.msq.sql.entity.ColumnNameAndTypes; +import org.apache.druid.msq.sql.entity.SqlStatementResult; +import org.apache.druid.query.QueryException; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.sql.calcite.planner.ColumnMappings; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +public class SqlStatementResourceHelper +{ + public static Optional> getSignature( + MSQControllerTask msqControllerTask + ) + { + // only populate signature for select q's + if (msqControllerTask.getQuerySpec().getDestination().getClass() == TaskReportMSQDestination.class) { + ColumnMappings columnMappings = msqControllerTask.getQuerySpec().getColumnMappings(); + List sqlTypeNames = msqControllerTask.getSqlTypeNames(); + if (sqlTypeNames == null || sqlTypeNames.size() != columnMappings.size()) { + return Optional.empty(); + } + List nativeTypeNames = msqControllerTask.getNativeTypeNames(); + if (nativeTypeNames == null || nativeTypeNames.size() != columnMappings.size()) { + return Optional.empty(); + } + List signature = new ArrayList<>(columnMappings.size()); + int index = 0; + for (String colName : columnMappings.getOutputColumnNames()) { + signature.add(new ColumnNameAndTypes( + colName, + sqlTypeNames.get(index).getName(), + nativeTypeNames.get(index).asTypeString() + )); + index++; + } + return Optional.of(signature); + } + return Optional.empty(); + } + + + public static void isMSQPayload(TaskPayloadResponse taskPayloadResponse, String queryId) throws DruidException + { + if (taskPayloadResponse == null || taskPayloadResponse.getPayload() == null) { + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build( + "Query[%s] not found", queryId); + } + + if (MSQControllerTask.class != taskPayloadResponse.getPayload().getClass()) { + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build( + "Query[%s] not found", queryId); + } + } + + public static SqlStatementState getSqlStatementState(TaskStatusPlus taskStatusPlus) + { + TaskState state = taskStatusPlus.getStatusCode(); + if (state == null) { + return SqlStatementState.ACCEPTED; + } + + switch (state) { + case FAILED: + return SqlStatementState.FAILED; + case RUNNING: + if (TaskLocation.unknown().equals(taskStatusPlus.getLocation())) { + return SqlStatementState.ACCEPTED; + } else { + return SqlStatementState.RUNNING; + } + case SUCCESS: + return SqlStatementState.SUCCESS; + default: + throw new ISE("Unrecognized state[%s] found.", state); + } + } + + @SuppressWarnings("unchecked") + + + public static long getLastIndex(Long numberOfRows, long start) + { + final long last; + if (numberOfRows == null) { + last = Long.MAX_VALUE; + } else { + long finalIndex; + try { + finalIndex = Math.addExact(start, numberOfRows); + } + catch (ArithmeticException e) { + finalIndex = Long.MAX_VALUE; + } + last = finalIndex; + } + return last; + } + + public static Optional> getRowsAndSizeFromPayload(Map payload, boolean isSelectQuery) + { + List stages = getList(payload, "stages"); + if (stages == null || stages.isEmpty()) { + return Optional.empty(); + } else { + int maxStage = stages.size() - 1; // Last stage output is the total number of rows returned to the end user. + Map counterMap = getMap(getMap(payload, "counters"), String.valueOf(maxStage)); + long rows = -1L; + long sizeInBytes = -1L; + if (counterMap == null) { + return Optional.empty(); + } + for (Map.Entry worker : counterMap.entrySet()) { + Object workerChannels = worker.getValue(); + if (workerChannels == null || !(workerChannels instanceof Map)) { + return Optional.empty(); + } + if (isSelectQuery) { + Object output = ((Map) workerChannels).get("output"); + if (output != null && output instanceof Map) { + List rowsPerChannel = (List) ((Map) output).get("rows"); + List bytesPerChannel = (List) ((Map) output).get("bytes"); + for (Integer row : rowsPerChannel) { + rows = rows + row; + } + for (Integer bytes : bytesPerChannel) { + sizeInBytes = sizeInBytes + bytes; + } + } + } else { + Object output = ((Map) workerChannels).get("segmentGenerationProgress"); + if (output != null && output instanceof Map) { + rows += (Integer) ((Map) output).get("rowsPushed"); + } + } + } + + return Optional.of(new Pair<>(rows == -1L ? null : rows + 1, sizeInBytes == -1L ? null : sizeInBytes + 1)); + } + } + + + public static Optional getExceptionPayload( + String queryId, + TaskStatusResponse taskResponse, + TaskStatusPlus statusPlus, + SqlStatementState sqlStatementState, + Map msqPayload + ) + { + Map exceptionDetails = getQueryExceptionDetails(getPayload(msqPayload)); + Map exception = getMap(exceptionDetails, "error"); + if (exceptionDetails == null || exception == null) { + return Optional.of(new SqlStatementResult( + queryId, + sqlStatementState, + taskResponse.getStatus().getCreatedTime(), + null, + taskResponse.getStatus().getDuration(), + null, + new QueryException(null, statusPlus.getErrorMsg(), null, null, null) + )); + } + + final String errorMessage = String.valueOf(exception.getOrDefault("errorMessage", statusPlus.getErrorMsg())); + exception.remove("errorMessage"); + String host = (String) exceptionDetails.getOrDefault("host", null); + Map stringException = new HashMap<>(); + for (Map.Entry exceptionKeys : exception.entrySet()) { + stringException.put(exceptionKeys.getKey(), String.valueOf(exceptionKeys.getValue())); + } + return Optional.of(new SqlStatementResult( + queryId, + sqlStatementState, + taskResponse.getStatus().getCreatedTime(), + null, + taskResponse.getStatus().getDuration(), + null, + new QueryException(null, errorMessage, null, host, stringException) + )); + } + + public static Map getQueryExceptionDetails(Map payload) + { + return getMap(getMap(payload, "status"), "errorReport"); + } + + public static Map getMap(Map map, String key) + { + if (map == null) { + return null; + } + return (Map) map.get(key); + } + + @SuppressWarnings("rawtypes") + public static List getList(Map map, String key) + { + if (map == null) { + return null; + } + return (List) map.get(key); + } + + /** + * Get results from report + */ + @SuppressWarnings("unchecked") + public static Optional> getResults(Map payload) + { + Map resultsHolder = getMap(payload, "results"); + + if (resultsHolder == null) { + return Optional.empty(); + } + + List data = (List) resultsHolder.get("results"); + List rows = new ArrayList<>(); + if (data != null) { + rows.addAll(data); + } + return Optional.of(rows); + } + + public static Map getPayload(Map results) + { + Map msqReport = getMap(results, "multiStageQuery"); + Map payload = getMap(msqReport, "payload"); + return payload; + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java index 21948cbe0527..e87834fe47bf 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java @@ -36,7 +36,6 @@ import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryException; import org.apache.druid.segment.column.ValueType; -import org.apache.druid.server.initialization.ServerConfig; import org.apache.druid.sql.calcite.util.CalciteTests; import org.apache.druid.sql.http.SqlQuery; import org.junit.Assert; @@ -49,7 +48,6 @@ public class SqlMsqStatementResourcePostTest extends MSQTestBase { - private SqlStatementResource resource; @Before @@ -57,7 +55,6 @@ public void init() { resource = new SqlStatementResource( sqlStatementFactory, - new ServerConfig(), CalciteTests.TEST_AUTHORIZER_MAPPER, objectMapper, indexingServiceClient diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java index db66baa24f73..8c1bf81a15f8 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java @@ -29,6 +29,7 @@ import org.apache.calcite.sql.type.SqlTypeName; import org.apache.druid.client.indexing.TaskPayloadResponse; import org.apache.druid.client.indexing.TaskStatusResponse; +import org.apache.druid.error.ErrorResponse; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatusPlus; @@ -60,7 +61,6 @@ import org.apache.druid.msq.test.MSQTestBase; import org.apache.druid.query.Druids; import org.apache.druid.query.Query; -import org.apache.druid.query.QueryException; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.rpc.indexing.OverlordClient; @@ -68,7 +68,6 @@ import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.ValueType; -import org.apache.druid.server.initialization.ServerConfig; import org.apache.druid.server.mocks.MockHttpServletRequest; import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthenticationResult; @@ -333,7 +332,6 @@ public void init() throws Exception setupMocks(overlordClient); resource = new SqlStatementResource( sqlStatementFactory, - new ServerConfig(), CalciteTests.TEST_AUTHORIZER_MAPPER, JSON_MAPPER, overlordClient @@ -614,10 +612,11 @@ public void testMSQSelectAcceptedQuery() assertExceptionMessage( resource.doGetResults(ACCEPTED_SELECT_MSQ_QUERY, null, null, makeOkRequest()), StringUtils.format( - "Query is [%s]. Please wait for it to complete.", + "Query[%s] is currently in [%s] state. Please wait for it to complete.", + ACCEPTED_SELECT_MSQ_QUERY, SqlStatementState.ACCEPTED ), - Response.Status.NOT_FOUND + Response.Status.BAD_REQUEST ); Assert.assertEquals( Response.Status.ACCEPTED.getStatusCode(), @@ -648,10 +647,11 @@ public void testMSQSelectRunningQuery() assertExceptionMessage( resource.doGetResults(RUNNING_SELECT_MSQ_QUERY, null, null, makeOkRequest()), StringUtils.format( - "Query is [%s]. Please wait for it to complete.", + "Query[%s] is currently in [%s] state. Please wait for it to complete.", + RUNNING_SELECT_MSQ_QUERY, SqlStatementState.RUNNING ), - Response.Status.NOT_FOUND + Response.Status.BAD_REQUEST ); Assert.assertEquals( Response.Status.ACCEPTED.getStatusCode(), @@ -726,11 +726,11 @@ public void testFinishedSelectMSQQuery() throws Exception ); Assert.assertEquals( - Response.Status.PRECONDITION_FAILED.getStatusCode(), + Response.Status.BAD_REQUEST.getStatusCode(), resource.doGetResults(FINISHED_SELECT_MSQ_QUERY, -1L, 3L, makeOkRequest()).getStatus() ); Assert.assertEquals( - Response.Status.PRECONDITION_FAILED.getStatusCode(), + Response.Status.BAD_REQUEST.getStatusCode(), resource.doGetResults(FINISHED_SELECT_MSQ_QUERY, null, -1L, makeOkRequest()).getStatus() ); @@ -744,8 +744,11 @@ public void testFailedMSQQuery() assertExceptionMessage(resource.doGetStatus(queryID, makeOkRequest()), FAILURE_MSG, Response.Status.OK); assertExceptionMessage( resource.doGetResults(queryID, null, null, makeOkRequest()), - FAILURE_MSG, - Response.Status.NOT_FOUND + StringUtils.format( + "Query[%s] failed. Hit status api for more details.", + queryID + ), + Response.Status.BAD_REQUEST ); Assert.assertEquals( @@ -795,11 +798,11 @@ public void testFinishedInsertMSQQuery() throws Exception ))); Assert.assertEquals( - Response.Status.PRECONDITION_FAILED.getStatusCode(), + Response.Status.BAD_REQUEST.getStatusCode(), resource.doGetResults(FINISHED_INSERT_MSQ_QUERY, -1L, 3L, makeOkRequest()).getStatus() ); Assert.assertEquals( - Response.Status.PRECONDITION_FAILED.getStatusCode(), + Response.Status.BAD_REQUEST.getStatusCode(), resource.doGetResults(FINISHED_INSERT_MSQ_QUERY, null, -1L, makeOkRequest()).getStatus() ); @@ -836,10 +839,11 @@ public void testMSQInsertAcceptedQuery() assertExceptionMessage( resource.doGetResults(ACCEPTED_INSERT_MSQ_TASK, null, null, makeOkRequest()), StringUtils.format( - "Query is [%s]. Please wait for it to complete.", + "Query[%s] is currently in [%s] state. Please wait for it to complete.", + ACCEPTED_INSERT_MSQ_TASK, SqlStatementState.ACCEPTED ), - Response.Status.NOT_FOUND + Response.Status.BAD_REQUEST ); Assert.assertEquals( Response.Status.ACCEPTED.getStatusCode(), @@ -868,10 +872,11 @@ public void testMSQInsertRunningQuery() assertExceptionMessage( resource.doGetResults(RUNNING_INSERT_MSQ_QUERY, null, null, makeOkRequest()), StringUtils.format( - "Query is [%s]. Please wait for it to complete.", + "Query[%s] is currently in [%s] state. Please wait for it to complete.", + RUNNING_INSERT_MSQ_QUERY, SqlStatementState.RUNNING ), - Response.Status.NOT_FOUND + Response.Status.BAD_REQUEST ); Assert.assertEquals( Response.Status.ACCEPTED.getStatusCode(), @@ -893,7 +898,7 @@ public static void assertExceptionMessage( ) { Assert.assertEquals(expectectedStatus.getStatusCode(), response.getStatus()); - Assert.assertEquals(exceptionMessage, getQueryExceptionFromResponse(response).getMessage()); + Assert.assertEquals(exceptionMessage, getQueryExceptionFromResponse(response)); } public static List getResultRowsFromResponse(Response resultsResponse) throws IOException @@ -905,12 +910,12 @@ public static List getResultRowsFromResponse(Response resultsResponse) throws IO return JSON_MAPPER.readValue(bytes, List.class); } - private static QueryException getQueryExceptionFromResponse(Response response) + private static String getQueryExceptionFromResponse(Response response) { if (response.getEntity() instanceof SqlStatementResult) { - return ((SqlStatementResult) response.getEntity()).getQueryException(); + return ((SqlStatementResult) response.getEntity()).getQueryException().getMessage(); } else { - return (QueryException) response.getEntity(); + return ((ErrorResponse) response.getEntity()).getUnderlyingException().getMessage(); } } From 06a7fb433f9e4327d1ae754a0e53bbe6e7315d12 Mon Sep 17 00:00:00 2001 From: cryptoe Date: Mon, 26 Jun 2023 22:39:48 +0530 Subject: [PATCH 09/15] Merge things --- .gitignore | 2 +- website/i18n/en.json | 790 ------------------------------------------- 2 files changed, 1 insertion(+), 791 deletions(-) delete mode 100644 website/i18n/en.json diff --git a/.gitignore b/.gitignore index a60eb68173a2..cd33e6271a1d 100644 --- a/.gitignore +++ b/.gitignore @@ -35,7 +35,7 @@ integration-tests/gen-scripts/ **/.local/ **/druidapi.egg-info/ examples/quickstart/jupyter-notebooks/docker-jupyter/notebooks - +website/i18n/* # ignore NetBeans IDE specific files nbproject nbactions.xml diff --git a/website/i18n/en.json b/website/i18n/en.json deleted file mode 100644 index 7e5249167e4f..000000000000 --- a/website/i18n/en.json +++ /dev/null @@ -1,790 +0,0 @@ -{ - "_comment": "This file is auto-generated by write-translations.js", - "localized-strings": { - "next": "Next", - "previous": "Previous", - "tagline": "A fast analytical database", - "docs": { - "comparisons/druid-vs-elasticsearch": { - "title": "Apache Druid vs Elasticsearch" - }, - "comparisons/druid-vs-key-value": { - "title": "Apache Druid vs. Key/Value Stores (HBase/Cassandra/OpenTSDB)" - }, - "comparisons/druid-vs-kudu": { - "title": "Apache Druid vs Kudu" - }, - "comparisons/druid-vs-redshift": { - "title": "Apache Druid vs Redshift" - }, - "comparisons/druid-vs-spark": { - "title": "Apache Druid vs Spark" - }, - "comparisons/druid-vs-sql-on-hadoop": { - "title": "Apache Druid vs SQL-on-Hadoop" - }, - "configuration/human-readable-byte": { - "title": "Human-readable Byte Configuration Reference" - }, - "configuration/index": { - "title": "Configuration reference" - }, - "configuration/logging": { - "title": "Logging" - }, - "data-management/automatic-compaction": { - "title": "Automatic compaction" - }, - "data-management/compaction": { - "title": "Compaction" - }, - "data-management/delete": { - "title": "Data deletion" - }, - "data-management/index": { - "title": "Data management", - "sidebar_label": "Overview" - }, - "data-management/schema-changes": { - "title": "Schema changes" - }, - "data-management/update": { - "title": "Data updates" - }, - "dependencies/deep-storage": { - "title": "Deep storage" - }, - "dependencies/metadata-storage": { - "title": "Metadata storage" - }, - "dependencies/zookeeper": { - "title": "ZooKeeper" - }, - "design/architecture": { - "title": "Design" - }, - "design/auth": { - "title": "Authentication and Authorization" - }, - "design/broker": { - "title": "Broker" - }, - "design/coordinator": { - "title": "Coordinator Process" - }, - "design/extensions-contrib/dropwizard": { - "title": "Dropwizard metrics emitter" - }, - "design/historical": { - "title": "Historical Process" - }, - "design/index": { - "title": "Introduction to Apache Druid" - }, - "design/indexer": { - "title": "Indexer Process" - }, - "design/indexing-service": { - "title": "Indexing Service" - }, - "design/middlemanager": { - "title": "MiddleManager Process" - }, - "design/overlord": { - "title": "Overlord Process" - }, - "design/peons": { - "title": "Peons" - }, - "design/processes": { - "title": "Processes and servers" - }, - "design/router": { - "title": "Router Process" - }, - "design/segments": { - "title": "Segments" - }, - "development/build": { - "title": "Build from source" - }, - "development/experimental-features": { - "title": "Experimental features" - }, - "development/experimental": { - "title": "Experimental features" - }, - "development/extensions-contrib/aliyun-oss": { - "title": "Aliyun OSS" - }, - "development/extensions-contrib/ambari-metrics-emitter": { - "title": "Ambari Metrics Emitter" - }, - "development/extensions-contrib/cassandra": { - "title": "Apache Cassandra" - }, - "development/extensions-contrib/cloudfiles": { - "title": "Rackspace Cloud Files" - }, - "development/extensions-contrib/compressed-big-decimal": { - "title": "Compressed Big Decimal" - }, - "development/extensions-contrib/distinctcount": { - "title": "DistinctCount Aggregator" - }, - "development/extensions-contrib/gce-extensions": { - "title": "GCE Extensions" - }, - "development/extensions-contrib/graphite": { - "title": "Graphite Emitter" - }, - "development/extensions-contrib/influx": { - "title": "InfluxDB Line Protocol Parser" - }, - "development/extensions-contrib/influxdb-emitter": { - "title": "InfluxDB Emitter" - }, - "development/extensions-contrib/k8s-jobs": { - "title": "MM-less Druid in K8s" - }, - "development/extensions-contrib/kafka-emitter": { - "title": "Kafka Emitter" - }, - "development/extensions-contrib/materialized-view": { - "title": "Materialized View" - }, - "development/extensions-contrib/momentsketch-quantiles": { - "title": "Moment Sketches for Approximate Quantiles module" - }, - "development/extensions-contrib/moving-average-query": { - "title": "Moving Average Query" - }, - "development/extensions-contrib/opentsdb-emitter": { - "title": "OpenTSDB Emitter" - }, - "development/extensions-contrib/prometheus": { - "title": "Prometheus Emitter" - }, - "development/extensions-contrib/redis-cache": { - "title": "Druid Redis Cache" - }, - "development/extensions-contrib/sqlserver": { - "title": "Microsoft SQLServer" - }, - "development/extensions-contrib/statsd": { - "title": "StatsD Emitter" - }, - "development/extensions-contrib/tdigestsketch-quantiles": { - "title": "T-Digest Quantiles Sketch module" - }, - "development/extensions-contrib/thrift": { - "title": "Thrift" - }, - "development/extensions-contrib/time-min-max": { - "title": "Timestamp Min/Max aggregators" - }, - "development/extensions-core/approximate-histograms": { - "title": "Approximate Histogram aggregators" - }, - "development/extensions-core/avro": { - "title": "Apache Avro" - }, - "development/extensions-core/azure": { - "title": "Microsoft Azure" - }, - "development/extensions-core/bloom-filter": { - "title": "Bloom Filter" - }, - "development/extensions-core/datasketches-extension": { - "title": "DataSketches extension" - }, - "development/extensions-core/datasketches-hll": { - "title": "DataSketches HLL Sketch module" - }, - "development/extensions-core/datasketches-kll": { - "title": "DataSketches KLL Sketch module" - }, - "development/extensions-core/datasketches-quantiles": { - "title": "DataSketches Quantiles Sketch module" - }, - "development/extensions-core/datasketches-theta": { - "title": "DataSketches Theta Sketch module" - }, - "development/extensions-core/datasketches-tuple": { - "title": "DataSketches Tuple Sketch module" - }, - "development/extensions-core/druid-aws-rds": { - "title": "Druid AWS RDS Module" - }, - "development/extensions-core/druid-basic-security": { - "title": "Basic Security" - }, - "development/extensions-core/druid-kerberos": { - "title": "Kerberos" - }, - "development/extensions-core/druid-lookups": { - "title": "Cached Lookup Module" - }, - "development/extensions-core/druid-pac4j": { - "title": "Druid pac4j based Security extension" - }, - "development/extensions-core/druid-ranger-security": { - "title": "Apache Ranger Security" - }, - "development/extensions-core/examples": { - "title": "Extension Examples" - }, - "development/extensions-core/google": { - "title": "Google Cloud Storage" - }, - "development/extensions-core/hdfs": { - "title": "HDFS" - }, - "development/extensions-core/kafka-extraction-namespace": { - "title": "Apache Kafka Lookups" - }, - "development/extensions-core/kafka-ingestion": { - "title": "Apache Kafka ingestion", - "sidebar_label": "Apache Kafka ingestion" - }, - "development/extensions-core/kafka-supervisor-operations": { - "title": "Apache Kafka supervisor operations reference", - "sidebar_label": "Apache Kafka operations" - }, - "development/extensions-core/kafka-supervisor-reference": { - "title": "Apache Kafka supervisor reference", - "sidebar_label": "Apache Kafka supervisor" - }, - "development/extensions-core/kinesis-ingestion": { - "title": "Amazon Kinesis ingestion", - "sidebar_label": "Amazon Kinesis" - }, - "development/extensions-core/kubernetes": { - "title": "Kubernetes" - }, - "development/extensions-core/lookups-cached-global": { - "title": "Globally Cached Lookups" - }, - "development/extensions-core/mysql": { - "title": "MySQL Metadata Store" - }, - "development/extensions-core/orc": { - "title": "ORC Extension" - }, - "development/extensions-core/parquet": { - "title": "Apache Parquet Extension" - }, - "development/extensions-core/postgresql": { - "title": "PostgreSQL Metadata Store" - }, - "development/extensions-core/protobuf": { - "title": "Protobuf" - }, - "development/extensions-core/s3": { - "title": "S3-compatible" - }, - "development/extensions-core/simple-client-sslcontext": { - "title": "Simple SSLContext Provider Module" - }, - "development/extensions-core/stats": { - "title": "Stats aggregator" - }, - "development/extensions-core/test-stats": { - "title": "Test Stats Aggregators" - }, - "development/extensions": { - "title": "Extensions" - }, - "development/geo": { - "title": "Spatial filters" - }, - "development/javascript": { - "title": "JavaScript programming guide", - "sidebar_label": "JavaScript functionality" - }, - "development/modules": { - "title": "Creating extensions" - }, - "development/overview": { - "title": "Developing on Apache Druid", - "sidebar_label": "Developing on Druid" - }, - "development/versioning": { - "title": "Versioning" - }, - "ingestion/data-formats": { - "title": "Data formats" - }, - "ingestion/data-model": { - "title": "Druid data model", - "sidebar_label": "Data model" - }, - "ingestion/faq": { - "title": "Ingestion troubleshooting FAQ", - "sidebar_label": "Troubleshooting FAQ" - }, - "ingestion/hadoop": { - "title": "Hadoop-based ingestion", - "sidebar_label": "Hadoop-based" - }, - "ingestion/index": { - "title": "Ingestion" - }, - "ingestion/ingestion-spec": { - "title": "Ingestion spec reference", - "sidebar_label": "Ingestion spec" - }, - "ingestion/migrate-from-firehose": { - "title": "Migrate from firehose to input source ingestion", - "sidebar_label": "Migrate from firehose" - }, - "ingestion/native-batch-firehose": { - "title": "Native batch ingestion with firehose (Deprecated)", - "sidebar_label": "Firehose (deprecated)" - }, - "ingestion/native-batch-input-sources": { - "title": "Native batch input sources", - "sidebar_label": "Native batch: input sources" - }, - "ingestion/native-batch-simple-task": { - "title": "Native batch simple task indexing", - "sidebar_label": "Native batch (simple)" - }, - "ingestion/native-batch": { - "title": "Native batch ingestion", - "sidebar_label": "Native batch" - }, - "ingestion/partitioning": { - "title": "Partitioning", - "sidebar_label": "Partitioning" - }, - "ingestion/rollup": { - "title": "Data rollup", - "sidebar_label": "Data rollup" - }, - "ingestion/schema-design": { - "title": "Schema design tips" - }, - "ingestion/standalone-realtime": { - "title": "Realtime Process" - }, - "ingestion/tasks": { - "title": "Task reference" - }, - "ingestion/tranquility": { - "title": "Tranquility" - }, - "misc/math-expr": { - "title": "Expressions" - }, - "misc/papers-and-talks": { - "title": "Papers" - }, - "multi-stage-query/api": { - "title": "SQL-based ingestion and multi-stage query task API", - "sidebar_label": "API" - }, - "multi-stage-query/concepts": { - "title": "SQL-based ingestion concepts", - "sidebar_label": "Key concepts" - }, - "multi-stage-query/examples": { - "title": "SQL-based ingestion query examples", - "sidebar_label": "Examples" - }, - "multi-stage-query/index": { - "title": "SQL-based ingestion", - "sidebar_label": "Overview" - }, - "multi-stage-query/known-issues": { - "title": "SQL-based ingestion known issues", - "sidebar_label": "Known issues" - }, - "multi-stage-query/reference": { - "title": "SQL-based ingestion reference", - "sidebar_label": "Reference" - }, - "multi-stage-query/security": { - "title": "SQL-based ingestion security", - "sidebar_label": "Security" - }, - "operations/alerts": { - "title": "Alerts" - }, - "operations/api-reference": { - "title": "API reference" - }, - "operations/auth-ldap": { - "title": "Configure LDAP authentication", - "sidebar_label": "LDAP auth" - }, - "operations/basic-cluster-tuning": { - "title": "Basic cluster tuning" - }, - "operations/clean-metadata-store": { - "title": "Automated cleanup for metadata records", - "sidebar_label": "Automated metadata cleanup" - }, - "operations/deep-storage-migration": { - "title": "Deep storage migration" - }, - "operations/dump-segment": { - "title": "dump-segment tool" - }, - "operations/dynamic-config-provider": { - "title": "Dynamic Config Providers" - }, - "operations/export-metadata": { - "title": "Export Metadata Tool" - }, - "operations/getting-started": { - "title": "Getting started with Apache Druid" - }, - "operations/high-availability": { - "title": "High availability" - }, - "operations/http-compression": { - "title": "HTTP compression" - }, - "operations/insert-segment-to-db": { - "title": "insert-segment-to-db tool" - }, - "operations/java": { - "title": "Java runtime" - }, - "operations/kubernetes": { - "title": "kubernetes" - }, - "operations/metadata-migration": { - "title": "Metadata Migration" - }, - "operations/metrics": { - "title": "Metrics" - }, - "operations/mixed-workloads": { - "title": "Configure Druid for mixed workloads", - "sidebar_label": "Mixed workloads" - }, - "operations/other-hadoop": { - "title": "Working with different versions of Apache Hadoop" - }, - "operations/password-provider": { - "title": "Password providers" - }, - "operations/pull-deps": { - "title": "pull-deps tool" - }, - "operations/python": { - "title": "Python Installation" - }, - "operations/request-logging": { - "title": "Request logging", - "sidebar_label": "Request logging" - }, - "operations/reset-cluster": { - "title": "reset-cluster tool" - }, - "operations/rolling-updates": { - "title": "Rolling updates" - }, - "operations/rule-configuration": { - "title": "Using rules to drop and retain data" - }, - "operations/security-overview": { - "title": "Security overview" - }, - "operations/security-user-auth": { - "title": "User authentication and authorization" - }, - "operations/segment-optimization": { - "title": "Segment size optimization" - }, - "operations/single-server": { - "title": "Single server deployment" - }, - "operations/tls-support": { - "title": "TLS support" - }, - "operations/use_sbt_to_build_fat_jar": { - "title": "Content for build.sbt" - }, - "operations/web-console": { - "title": "Web console" - }, - "querying/aggregations": { - "title": "Aggregations" - }, - "querying/caching": { - "title": "Query caching" - }, - "querying/datasource": { - "title": "Datasources" - }, - "querying/datasourcemetadataquery": { - "title": "DatasourceMetadata queries", - "sidebar_label": "DatasourceMetadata" - }, - "querying/dimensionspecs": { - "title": "Query dimensions", - "sidebar_label": "Dimensions" - }, - "querying/filters": { - "title": "Query filters", - "sidebar_label": "Filters" - }, - "querying/granularities": { - "title": "Query granularities", - "sidebar_label": "Granularities" - }, - "querying/groupbyquery": { - "title": "GroupBy queries", - "sidebar_label": "GroupBy" - }, - "querying/having": { - "title": "Having filters (groupBy)" - }, - "querying/hll-old": { - "title": "Cardinality/HyperUnique aggregators" - }, - "querying/joins": { - "title": "Joins" - }, - "querying/limitspec": { - "title": "Sorting and limiting (groupBy)" - }, - "querying/lookups": { - "title": "Lookups" - }, - "querying/multi-value-dimensions": { - "title": "Multi-value dimensions" - }, - "querying/multitenancy": { - "title": "Multitenancy considerations", - "sidebar_label": "Multitenancy" - }, - "querying/nested-columns": { - "title": "Nested columns", - "sidebar_label": "Nested columns" - }, - "querying/post-aggregations": { - "title": "Post-aggregations" - }, - "querying/query-context": { - "title": "Query context", - "sidebar_label": "Query context" - }, - "querying/query-execution": { - "title": "Query execution" - }, - "querying/querying": { - "title": "Native queries" - }, - "querying/scan-query": { - "title": "Scan queries", - "sidebar_label": "Scan" - }, - "querying/searchquery": { - "title": "Search queries", - "sidebar_label": "Search" - }, - "querying/segmentmetadataquery": { - "title": "SegmentMetadata queries", - "sidebar_label": "SegmentMetadata" - }, - "querying/select-query": { - "title": "Select queries", - "sidebar_label": "Select" - }, - "querying/sorting-orders": { - "title": "String comparators" - }, - "querying/sql-aggregations": { - "title": "SQL aggregation functions", - "sidebar_label": "Aggregation functions" - }, - "querying/sql-api": { - "title": "Druid SQL API", - "sidebar_label": "Druid SQL API" - }, - "querying/sql-data-types": { - "title": "SQL data types", - "sidebar_label": "SQL data types" - }, - "querying/sql-functions": { - "title": "All Druid SQL functions", - "sidebar_label": "All functions" - }, - "querying/sql-jdbc": { - "title": "SQL JDBC driver API", - "sidebar_label": "JDBC driver API" - }, - "querying/sql-json-functions": { - "title": "SQL JSON functions", - "sidebar_label": "JSON functions" - }, - "querying/sql-metadata-tables": { - "title": "SQL metadata tables", - "sidebar_label": "SQL metadata tables" - }, - "querying/sql-multivalue-string-functions": { - "title": "SQL multi-value string functions", - "sidebar_label": "Multi-value string functions" - }, - "querying/sql-operators": { - "title": "Druid SQL Operators", - "sidebar_label": "Operators" - }, - "querying/sql-query-context": { - "title": "SQL query context", - "sidebar_label": "SQL query context" - }, - "querying/sql-scalar": { - "title": "SQL scalar functions", - "sidebar_label": "Scalar functions" - }, - "querying/sql-translation": { - "title": "SQL query translation", - "sidebar_label": "SQL query translation" - }, - "querying/sql": { - "title": "Druid SQL overview", - "sidebar_label": "Overview and syntax" - }, - "querying/timeboundaryquery": { - "title": "TimeBoundary queries", - "sidebar_label": "TimeBoundary" - }, - "querying/timeseriesquery": { - "title": "Timeseries queries", - "sidebar_label": "Timeseries" - }, - "querying/topnmetricspec": { - "title": "Sorting (topN)" - }, - "querying/topnquery": { - "title": "TopN queries", - "sidebar_label": "TopN" - }, - "querying/troubleshooting": { - "title": "Troubleshooting query execution in Druid", - "sidebar_label": "Troubleshooting" - }, - "querying/using-caching": { - "title": "Using query caching" - }, - "querying/virtual-columns": { - "title": "Virtual columns" - }, - "tutorials/cluster": { - "title": "Clustered deployment" - }, - "tutorials/docker": { - "title": "Tutorial: Run with Docker" - }, - "tutorials/index": { - "title": "Quickstart (local)" - }, - "tutorials/tutorial-batch-hadoop": { - "title": "Tutorial: Load batch data using Apache Hadoop", - "sidebar_label": "Load from Apache Hadoop" - }, - "tutorials/tutorial-batch-native": { - "title": "Load data with native batch ingestion" - }, - "tutorials/tutorial-batch": { - "title": "Tutorial: Loading a file", - "sidebar_label": "Load files natively" - }, - "tutorials/tutorial-compaction": { - "title": "Tutorial: Compacting segments", - "sidebar_label": "Compacting segments" - }, - "tutorials/tutorial-delete-data": { - "title": "Tutorial: Deleting data", - "sidebar_label": "Deleting data" - }, - "tutorials/tutorial-ingestion-spec": { - "title": "Tutorial: Writing an ingestion spec", - "sidebar_label": "Writing an ingestion spec" - }, - "tutorials/tutorial-jdbc": { - "title": "Tutorial: Using the JDBC driver to query Druid", - "sidebar_label": "JDBC connector" - }, - "tutorials/tutorial-jupyter-index": { - "title": "Jupyter Notebook tutorials" - }, - "tutorials/tutorial-kafka": { - "title": "Tutorial: Load streaming data from Apache Kafka", - "sidebar_label": "Load from Apache Kafka" - }, - "tutorials/tutorial-kerberos-hadoop": { - "title": "Configuring Apache Druid to use Kerberized Apache Hadoop as deep storage", - "sidebar_label": "Kerberized HDFS deep storage" - }, - "tutorials/tutorial-msq-convert-spec": { - "title": "Tutorial: Convert an ingestion spec for SQL-based ingestion", - "sidebar_label": "Convert ingestion spec to SQL" - }, - "tutorials/tutorial-msq-extern": { - "title": "Tutorial: Load files with SQL-based ingestion", - "sidebar_label": "Load files using SQL 🆕" - }, - "tutorials/tutorial-query": { - "title": "Tutorial: Querying data", - "sidebar_label": "Querying data" - }, - "tutorials/tutorial-retention": { - "title": "Tutorial: Configuring data retention", - "sidebar_label": "Configuring data retention" - }, - "tutorials/tutorial-rollup": { - "title": "Tutorial: Roll-up", - "sidebar_label": "Roll-up" - }, - "tutorials/tutorial-sketches-theta": { - "title": "Approximations with Theta sketches", - "sidebar_label": "Theta sketches" - }, - "tutorials/tutorial-transform-spec": { - "title": "Tutorial: Transforming input data", - "sidebar_label": "Transforming input data" - }, - "tutorials/tutorial-unnest-datasource": { - "title": "Tutorial: Unnest data in a column", - "sidebar_label": "Using the unnest datasource" - }, - "tutorials/tutorial-update-data": { - "title": "Tutorial: Updating existing data", - "sidebar_label": "Updating existing data" - } - }, - "links": { - "Technology": "Technology", - "Use Cases": "Use Cases", - "Powered By": "Powered By", - "Docs": "Docs", - "Community": "Community", - "Apache": "Apache", - "Download": "Download" - }, - "categories": { - "Getting started": "Getting started", - "Tutorials": "Tutorials", - "Design": "Design", - "Ingestion": "Ingestion", - "Data management": "Data management", - "Querying": "Querying", - "Configuration": "Configuration", - "Operations": "Operations", - "Development": "Development", - "Misc": "Misc", - "Hidden": "Hidden" - } - }, - "pages-strings": { - "Help Translate|recruit community translators for your project": "Help Translate", - "Edit this Doc|recruitment message asking to edit the doc source": "Edit", - "Translate this Doc|recruitment message asking to translate the docs": "Translate" - } -} From b423b380ca094afa1382f6b651b6252f6830554a Mon Sep 17 00:00:00 2001 From: cryptoe Date: Tue, 27 Jun 2023 20:25:17 +0530 Subject: [PATCH 10/15] Changing SqlStatementResult to use ErrorResponse --- .../msq/sql/entity/ResultSetInformation.java | 11 +- .../msq/sql/entity/SqlStatementResult.java | 37 +++-- .../msq/util/SqlStatementResourceHelper.java | 21 ++- .../sql/SqlMsqStatementResourcePostTest.java | 58 +++++-- .../msq/sql/SqlStatementResourceTest.java | 152 ++++++++---------- .../sql/entity/ColumnNameAndTypesTest.java | 53 ++++++ .../sql/entity/ResultSetInformationTest.java | 58 +++++++ .../sql/entity/SqlStatementResultTest.java | 95 +++++++++++ .../apache/druid/msq/test/MSQTestBase.java | 13 +- 9 files changed, 373 insertions(+), 125 deletions(-) create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/ColumnNameAndTypesTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/ResultSetInformationTest.java create mode 100644 extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/SqlStatementResultTest.java diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ResultSetInformation.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ResultSetInformation.java index 0ecfc17c0853..43201fdac6e0 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ResultSetInformation.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/ResultSetInformation.java @@ -36,9 +36,9 @@ public class ResultSetInformation @Nullable private final Long sizeInBytes; + @Nullable private final ResultFormat resultFormat; - @Nullable private final List records; @@ -47,11 +47,10 @@ public class ResultSetInformation @JsonCreator public ResultSetInformation( - @Nullable - @JsonProperty ResultFormat resultFormat, - @JsonProperty @Nullable Long numRows, - @JsonProperty @Nullable Long sizeInBytes, - @JsonProperty @Nullable String dataSource, + @JsonProperty("resultFormat") @Nullable ResultFormat resultFormat, + @JsonProperty("numRows") @Nullable Long numRows, + @JsonProperty("sizeInBytes") @Nullable Long sizeInBytes, + @JsonProperty("dataSource") @Nullable String dataSource, @JsonProperty("sampleRecords") @Nullable List records ) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/SqlStatementResult.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/SqlStatementResult.java index 967f4471becc..a0bfc7f2694b 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/SqlStatementResult.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/SqlStatementResult.java @@ -22,8 +22,8 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.error.ErrorResponse; import org.apache.druid.msq.sql.SqlStatementState; -import org.apache.druid.query.QueryException; import org.joda.time.DateTime; import javax.annotation.Nullable; @@ -49,7 +49,7 @@ public class SqlStatementResult private final ResultSetInformation resultSetInformation; @Nullable - private final QueryException queryException; + private final ErrorResponse errorResponse; @JsonCreator @@ -66,8 +66,8 @@ public SqlStatementResult( Long durationMs, @Nullable @JsonProperty("result") ResultSetInformation resultSetInformation, - @Nullable @JsonProperty("exception") - QueryException queryException + @Nullable @JsonProperty("errorDetails") + ErrorResponse errorResponse ) { @@ -77,7 +77,7 @@ public SqlStatementResult( this.sqlRowSignature = sqlRowSignature; this.durationMs = durationMs; this.resultSetInformation = resultSetInformation; - this.queryException = queryException; + this.errorResponse = errorResponse; } @JsonProperty @@ -114,7 +114,7 @@ public Long getDurationMs() return durationMs; } - @JsonProperty + @JsonProperty("result") @Nullable @JsonInclude(JsonInclude.Include.NON_NULL) public ResultSetInformation getResultSetInformation() @@ -122,14 +122,15 @@ public ResultSetInformation getResultSetInformation() return resultSetInformation; } - @JsonProperty + @JsonProperty("errorDetails") @Nullable @JsonInclude(JsonInclude.Include.NON_NULL) - public QueryException getQueryException() + public ErrorResponse getErrorResponse() { - return queryException; + return errorResponse; } + @Override public boolean equals(Object o) { @@ -147,15 +148,23 @@ public boolean equals(Object o) durationMs, that.durationMs ) && Objects.equals(resultSetInformation, that.resultSetInformation) && Objects.equals( - queryException, - that.queryException + errorResponse == null ? null : errorResponse.getAsMap(), + that.errorResponse == null ? null : errorResponse.getAsMap() ); } @Override public int hashCode() { - return Objects.hash(queryId, state, createdAt, sqlRowSignature, durationMs, resultSetInformation, queryException); + return Objects.hash( + queryId, + state, + createdAt, + sqlRowSignature, + durationMs, + resultSetInformation, + errorResponse == null ? null : errorResponse.getAsMap() + ); } @Override @@ -168,7 +177,9 @@ public String toString() ", sqlRowSignature=" + sqlRowSignature + ", durationInMs=" + durationMs + ", resultSetInformation=" + resultSetInformation + - ", queryException=" + queryException + + ", errorResponse=" + (errorResponse == null + ? "{}" + : errorResponse.getAsMap().toString()) + '}'; } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/SqlStatementResourceHelper.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/SqlStatementResourceHelper.java index ba15b927ca33..4c0474e1f93f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/SqlStatementResourceHelper.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/SqlStatementResourceHelper.java @@ -34,7 +34,6 @@ import org.apache.druid.msq.sql.SqlStatementState; import org.apache.druid.msq.sql.entity.ColumnNameAndTypes; import org.apache.druid.msq.sql.entity.SqlStatementResult; -import org.apache.druid.query.QueryException; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.sql.calcite.planner.ColumnMappings; @@ -199,13 +198,16 @@ public static Optional getExceptionPayload( null, taskResponse.getStatus().getDuration(), null, - new QueryException(null, statusPlus.getErrorMsg(), null, null, null) + DruidException.forPersona(DruidException.Persona.DEVELOPER) + .ofCategory(DruidException.Category.UNCATEGORIZED) + .build(taskResponse.getStatus().getErrorMsg()).toErrorResponse() )); } final String errorMessage = String.valueOf(exception.getOrDefault("errorMessage", statusPlus.getErrorMsg())); exception.remove("errorMessage"); - String host = (String) exceptionDetails.getOrDefault("host", null); + String errorCode = String.valueOf(exception.getOrDefault("errorCode", "unknown")); + exception.remove("errorCode"); Map stringException = new HashMap<>(); for (Map.Entry exceptionKeys : exception.entrySet()) { stringException.put(exceptionKeys.getKey(), String.valueOf(exceptionKeys.getValue())); @@ -217,7 +219,18 @@ public static Optional getExceptionPayload( null, taskResponse.getStatus().getDuration(), null, - new QueryException(null, errorMessage, null, host, stringException) + DruidException.fromFailure(new DruidException.Failure(errorCode) + { + @Override + protected DruidException makeException(DruidException.DruidExceptionBuilder bob) + { + DruidException ex = bob.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.UNCATEGORIZED) + .build(errorMessage); + ex.withContext(stringException); + return ex; + } + }).toErrorResponse() )); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java index e87834fe47bf..7729e306deed 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java @@ -21,11 +21,15 @@ import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.msq.indexing.MSQControllerTask; import org.apache.druid.msq.indexing.error.InsertCannotBeEmptyFault; +import org.apache.druid.msq.indexing.error.MSQException; import org.apache.druid.msq.sql.entity.ColumnNameAndTypes; import org.apache.druid.msq.sql.entity.ResultSetInformation; import org.apache.druid.msq.sql.entity.SqlStatementResult; @@ -34,7 +38,6 @@ import org.apache.druid.msq.test.MSQTestOverlordServiceClient; import org.apache.druid.query.ExecutionMode; import org.apache.druid.query.QueryContexts; -import org.apache.druid.query.QueryException; import org.apache.druid.segment.column.ValueType; import org.apache.druid.sql.calcite.util.CalciteTests; import org.apache.druid.sql.http.SqlQuery; @@ -49,6 +52,7 @@ public class SqlMsqStatementResourcePostTest extends MSQTestBase { private SqlStatementResource resource; + public static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); @Before public void init() @@ -82,7 +86,7 @@ public void testMSQSelectQueryTest() throws IOException false, false, false, - ImmutableMap.of(QueryContexts.CTX_EXECUTION_MODE, ExecutionMode.ASYNC.name()), + defaultAsyncContext(), null ), SqlStatementResourceTest.makeOkRequest()); @@ -136,7 +140,7 @@ public void insertCannotBeEmptyFaultTest() false, false, false, - ImmutableMap.of(QueryContexts.CTX_EXECUTION_MODE, ExecutionMode.ASYNC.name()), + defaultAsyncContext(), null ), SqlStatementResourceTest.makeOkRequest()); Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); @@ -145,6 +149,8 @@ public void insertCannotBeEmptyFaultTest() InsertCannotBeEmptyFault insertCannotBeEmptyFault = new InsertCannotBeEmptyFault("foo1"); + MSQException insertCannotBeEmpty = new MSQException(insertCannotBeEmptyFault); + SqlStatementResult expected = new SqlStatementResult( actual.getQueryId(), SqlStatementState.FAILED, @@ -152,21 +158,43 @@ public void insertCannotBeEmptyFaultTest() null, MSQTestOverlordServiceClient.DURATION, null, - new QueryException( - null, - insertCannotBeEmptyFault.getErrorMessage(), - null, - "localhost:8080", - ImmutableMap.of( - "errorCode", - InsertCannotBeEmptyFault.CODE, - "dataSource", - insertCannotBeEmptyFault.getDataSource() - ) - ) + DruidException.fromFailure(new DruidException.Failure(InsertCannotBeEmptyFault.CODE) + { + @Override + protected DruidException makeException(DruidException.DruidExceptionBuilder bob) + { + DruidException e = bob.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.UNCATEGORIZED) + .build(insertCannotBeEmpty.getMessage()); + e.withContext("dataSource", insertCannotBeEmptyFault.getDataSource()); + return e; + } + }).toErrorResponse() ); Assert.assertEquals(expected, actual); } + private static ImmutableMap defaultAsyncContext() + { + return ImmutableMap.of(QueryContexts.CTX_EXECUTION_MODE, ExecutionMode.ASYNC.name()); + } + + @Test + public void forbiddenTest() + { + Assert.assertEquals(Response.Status.FORBIDDEN.getStatusCode(), resource.doPost( + new SqlQuery( + StringUtils.format("select * from %s", CalciteTests.FORBIDDEN_DATASOURCE), + null, + false, + false, + false, + defaultAsyncContext(), + null + ), + SqlStatementResourceTest.makeOkRequest() + ).getStatus()); + } + } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java index 8c1bf81a15f8..4002d3614528 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.api.client.util.Maps; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.util.concurrent.Futures; @@ -88,6 +87,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -95,9 +95,8 @@ public class SqlStatementResourceTest extends MSQTestBase { + public static final DateTime CREATED_TIME = DateTimes.of("2023-05-31T12:00Z"); private static final ObjectMapper JSON_MAPPER = TestHelper.makeJsonMapper(); - private static SqlStatementResource resource; - private static final String ACCEPTED_SELECT_MSQ_QUERY = "QUERY_ID_1"; private static final String RUNNING_SELECT_MSQ_QUERY = "QUERY_ID_2"; private static final String FINISHED_SELECT_MSQ_QUERY = "QUERY_ID_3"; @@ -158,7 +157,7 @@ public class SqlStatementResourceTest extends MSQTestBase MSQTuningConfig.defaultConfig()) .build(), "select _time,alias,market from test", - Maps.newHashMap(), + new HashMap<>(), null, ImmutableList.of( SqlTypeName.TIMESTAMP, @@ -203,7 +202,7 @@ public class SqlStatementResourceTest extends MSQTestBase MSQTuningConfig.defaultConfig()) .build(), "insert into test select _time,alias,market from test", - Maps.newHashMap(), + new HashMap<>(), null, ImmutableList.of( SqlTypeName.TIMESTAMP, @@ -294,18 +293,10 @@ public class SqlStatementResourceTest extends MSQTestBase null ) ); - - @Mock - private static OverlordClient overlordClient; - - private static final DateTime CREATED_TIME = DateTimes.of("2023-05-31T12:00Z"); private static final DateTime QUEUE_INSERTION_TIME = DateTimes.of("2023-05-31T12:01Z"); - - private static final Map ROW1 = ImmutableMap.of("_time", 123, "alias", "foo", "market", "bar"); private static final Map ROW2 = ImmutableMap.of("_time", 234, "alias", "foo1", "market", "bar1"); - - private static final ImmutableList COL_NAME_AND_TYPES = ImmutableList.of( + public static final ImmutableList COL_NAME_AND_TYPES = ImmutableList.of( new ColumnNameAndTypes( "_time", SqlTypeName.TIMESTAMP.getName(), @@ -322,21 +313,10 @@ public class SqlStatementResourceTest extends MSQTestBase ValueType.STRING.name() ) ); - private static final String FAILURE_MSG = "failure msg"; - - @Before - public void init() throws Exception - { - overlordClient = Mockito.mock(OverlordClient.class); - setupMocks(overlordClient); - resource = new SqlStatementResource( - sqlStatementFactory, - CalciteTests.TEST_AUTHORIZER_MAPPER, - JSON_MAPPER, - overlordClient - ); - } + private static SqlStatementResource resource; + @Mock + private static OverlordClient overlordClient; private static void setupMocks(OverlordClient indexingServiceClient) throws JsonProcessingException { @@ -591,6 +571,66 @@ private static void setupMocks(OverlordClient indexingServiceClient) throws Json } + public static void assertNullResponse(Response response, Response.Status expectectedStatus) + { + Assert.assertEquals(expectectedStatus.getStatusCode(), response.getStatus()); + Assert.assertNull(response.getEntity()); + } + + public static void assertExceptionMessage( + Response response, + String exceptionMessage, + Response.Status expectectedStatus + ) + { + Assert.assertEquals(expectectedStatus.getStatusCode(), response.getStatus()); + Assert.assertEquals(exceptionMessage, getQueryExceptionFromResponse(response)); + } + + public static List getResultRowsFromResponse(Response resultsResponse) throws IOException + { + byte[] bytes = SqlResourceTest.responseToByteArray(resultsResponse); + if (bytes == null) { + return null; + } + return JSON_MAPPER.readValue(bytes, List.class); + } + + private static String getQueryExceptionFromResponse(Response response) + { + if (response.getEntity() instanceof SqlStatementResult) { + return ((SqlStatementResult) response.getEntity()).getErrorResponse().getUnderlyingException().getMessage(); + } else { + return ((ErrorResponse) response.getEntity()).getUnderlyingException().getMessage(); + } + } + + public static MockHttpServletRequest makeOkRequest() + { + return makeExpectedReq(CalciteTests.REGULAR_USER_AUTH_RESULT); + } + + public static MockHttpServletRequest makeExpectedReq(AuthenticationResult authenticationResult) + { + MockHttpServletRequest req = new MockHttpServletRequest(); + req.attributes.put(AuthConfig.DRUID_AUTHENTICATION_RESULT, authenticationResult); + req.remoteAddr = "1.2.3.4"; + return req; + } + + @Before + public void init() throws Exception + { + overlordClient = Mockito.mock(OverlordClient.class); + setupMocks(overlordClient); + resource = new SqlStatementResource( + sqlStatementFactory, + CalciteTests.TEST_AUTHORIZER_MAPPER, + JSON_MAPPER, + overlordClient + ); + } + @Test public void testMSQSelectAcceptedQuery() { @@ -659,7 +699,6 @@ public void testMSQSelectRunningQuery() ); } - @Test public void testFinishedSelectMSQQuery() throws Exception { @@ -736,7 +775,6 @@ public void testFinishedSelectMSQQuery() throws Exception } - @Test public void testFailedMSQQuery() { @@ -758,7 +796,6 @@ public void testFailedMSQQuery() } } - @Test public void testFinishedInsertMSQQuery() throws Exception { @@ -884,56 +921,9 @@ public void testMSQInsertRunningQuery() ); } - - public static void assertNullResponse(Response response, Response.Status expectectedStatus) - { - Assert.assertEquals(expectectedStatus.getStatusCode(), response.getStatus()); - Assert.assertNull(response.getEntity()); - } - - public static void assertExceptionMessage( - Response response, - String exceptionMessage, - Response.Status expectectedStatus - ) - { - Assert.assertEquals(expectectedStatus.getStatusCode(), response.getStatus()); - Assert.assertEquals(exceptionMessage, getQueryExceptionFromResponse(response)); - } - - public static List getResultRowsFromResponse(Response resultsResponse) throws IOException - { - byte[] bytes = SqlResourceTest.responseToByteArray(resultsResponse); - if (bytes == null) { - return null; - } - return JSON_MAPPER.readValue(bytes, List.class); - } - - private static String getQueryExceptionFromResponse(Response response) - { - if (response.getEntity() instanceof SqlStatementResult) { - return ((SqlStatementResult) response.getEntity()).getQueryException().getMessage(); - } else { - return ((ErrorResponse) response.getEntity()).getUnderlyingException().getMessage(); - } - } - - public static MockHttpServletRequest makeOkRequest() - { - return makeExpectedReq(CalciteTests.REGULAR_USER_AUTH_RESULT); - } - - public static MockHttpServletRequest makeOtherUserRequest() - { - return makeExpectedReq(CalciteTests.SUPER_USER_AUTH_RESULT); - } - - public static MockHttpServletRequest makeExpectedReq(AuthenticationResult authenticationResult) + @Test + public void testIsEnabled() { - MockHttpServletRequest req = new MockHttpServletRequest(); - req.attributes.put(AuthConfig.DRUID_AUTHENTICATION_RESULT, authenticationResult); - req.remoteAddr = "1.2.3.4"; - return req; + Assert.assertEquals(Response.Status.OK.getStatusCode(), resource.isEnabled(makeOkRequest()).getStatus()); } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/ColumnNameAndTypesTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/ColumnNameAndTypesTest.java new file mode 100644 index 000000000000..f0524849995f --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/ColumnNameAndTypesTest.java @@ -0,0 +1,53 @@ +/* + * 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.msq.sql.entity; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.junit.Assert; +import org.junit.Test; + +public class ColumnNameAndTypesTest +{ + public static final ObjectMapper MAPPER = new ObjectMapper(); + + public static final ColumnNameAndTypes COLUMN_NAME_AND_TYPES = new ColumnNameAndTypes("test", "test1", "test2"); + public static final String JSON_STRING = "{\"name\":\"test\",\"type\":\"test1\",\"nativeType\":\"test2\"}"; + + @Test + public void sanityTest() throws JsonProcessingException + { + Assert.assertEquals(JSON_STRING, MAPPER.writeValueAsString(COLUMN_NAME_AND_TYPES)); + Assert.assertEquals( + COLUMN_NAME_AND_TYPES, + MAPPER.readValue(MAPPER.writeValueAsString(COLUMN_NAME_AND_TYPES), ColumnNameAndTypes.class) + ); + + Assert.assertEquals( + COLUMN_NAME_AND_TYPES.hashCode(), + MAPPER.readValue(MAPPER.writeValueAsString(COLUMN_NAME_AND_TYPES), ColumnNameAndTypes.class) + .hashCode() + ); + Assert.assertEquals("ColumnNameAndTypes{colName='test', sqlTypeName='test1', nativeTypeName='test2'}", + COLUMN_NAME_AND_TYPES.toString()); + + } + +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/ResultSetInformationTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/ResultSetInformationTest.java new file mode 100644 index 000000000000..0a186c646c0b --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/ResultSetInformationTest.java @@ -0,0 +1,58 @@ +/* + * 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.msq.sql.entity; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import org.apache.druid.sql.http.ResultFormat; +import org.junit.Assert; +import org.junit.Test; + +public class ResultSetInformationTest +{ + public static final ObjectMapper MAPPER = new ObjectMapper(); + + public static final ResultSetInformation RESULTS = new ResultSetInformation(ResultFormat.OBJECT, 1l, 1l, "ds", + ImmutableList.of( + ImmutableList.of("1"), + ImmutableList.of("2"), + ImmutableList.of("3") + ) + ); + public static final String JSON_STRING = "{\"resultFormat\":\"object\",\"numRows\":1,\"sizeInBytes\":1,\"dataSource\":\"ds\",\"sampleRecords\":[[\"1\"],[\"2\"],[\"3\"]]}"; + + + @Test + public void sanityTest() throws JsonProcessingException + { + Assert.assertEquals(JSON_STRING, MAPPER.writeValueAsString(RESULTS)); + Assert.assertEquals(RESULTS, MAPPER.readValue(MAPPER.writeValueAsString(RESULTS), ResultSetInformation.class)); + Assert.assertEquals( + RESULTS.hashCode(), + MAPPER.readValue(MAPPER.writeValueAsString(RESULTS), ResultSetInformation.class).hashCode() + ); + Assert.assertEquals( + "ResultSetInformation{totalRows=1, totalSize=1, resultFormat=object, records=[[1], [2], [3]], dataSource='ds'}", + RESULTS.toString() + ); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/SqlStatementResultTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/SqlStatementResultTest.java new file mode 100644 index 000000000000..a0be3afcf74a --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/SqlStatementResultTest.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.msq.sql.entity; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.error.DruidException; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.msq.indexing.error.MSQException; +import org.apache.druid.msq.indexing.error.QueryNotSupportedFault; +import org.apache.druid.msq.sql.SqlStatementResourceTest; +import org.apache.druid.msq.sql.SqlStatementState; +import org.junit.Assert; +import org.junit.Test; + +public class SqlStatementResultTest +{ + public static final MSQException MSQ_EXCEPTION = new MSQException( + QueryNotSupportedFault.instance()); + + public static final ObjectMapper MAPPER = DefaultObjectMapper.INSTANCE; + + public static final String JSON_STRING = "{\"queryId\":\"q1\"," + + "\"state\":\"RUNNING\"," + + "\"createdAt\":\"2023-05-31T12:00:00.000Z\"," + + "\"schema\":[{\"name\":\"_time\",\"type\":\"TIMESTAMP\",\"nativeType\":\"LONG\"},{\"name\":\"alias\",\"type\":\"VARCHAR\",\"nativeType\":\"STRING\"},{\"name\":\"market\",\"type\":\"VARCHAR\",\"nativeType\":\"STRING\"}]," + + "\"durationMs\":100," + + "\"result\":{\"resultFormat\":\"object\",\"numRows\":1,\"sizeInBytes\":1,\"dataSource\":\"ds\",\"sampleRecords\":[[\"1\"],[\"2\"],[\"3\"]]}," + + "\"errorDetails\":{\"error\":\"druidException\",\"errorCode\":\"QueryNotSupported\",\"persona\":\"USER\",\"category\":\"UNCATEGORIZED\",\"errorMessage\":\"QueryNotSupported\",\"context\":{}}}"; + + public static final SqlStatementResult SQL_STATEMENT_RESULT = new SqlStatementResult( + "q1", + SqlStatementState.RUNNING, + SqlStatementResourceTest.CREATED_TIME, + SqlStatementResourceTest.COL_NAME_AND_TYPES, + 100L, + ResultSetInformationTest.RESULTS, + DruidException.fromFailure(new DruidException.Failure(MSQ_EXCEPTION.getFault().getErrorCode()) + { + @Override + protected DruidException makeException(DruidException.DruidExceptionBuilder bob) + { + DruidException ex = bob.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.UNCATEGORIZED) + .build(MSQ_EXCEPTION.getMessage()); + return ex; + } + }).toErrorResponse() + ); + + + @Test + public void sanityTest() throws JsonProcessingException + { + + Assert.assertEquals(JSON_STRING, MAPPER.writeValueAsString(SQL_STATEMENT_RESULT)); + Assert.assertEquals( + SQL_STATEMENT_RESULT, + MAPPER.readValue(MAPPER.writeValueAsString(SQL_STATEMENT_RESULT), SqlStatementResult.class) + ); + Assert.assertEquals( + SQL_STATEMENT_RESULT.hashCode(), + MAPPER.readValue(MAPPER.writeValueAsString(SQL_STATEMENT_RESULT), SqlStatementResult.class).hashCode() + ); + Assert.assertEquals( + "SqlStatementResult{" + + "queryId='q1'," + + " state=RUNNING," + + " createdAt=2023-05-31T12:00:00.000Z," + + " sqlRowSignature=[ColumnNameAndTypes{colName='_time', sqlTypeName='TIMESTAMP', nativeTypeName='LONG'}, ColumnNameAndTypes{colName='alias', sqlTypeName='VARCHAR', nativeTypeName='STRING'}, ColumnNameAndTypes{colName='market', sqlTypeName='VARCHAR', nativeTypeName='STRING'}]," + + " durationInMs=100," + + " resultSetInformation=ResultSetInformation{totalRows=1, totalSize=1, resultFormat=object, records=[[1], [2], [3]], dataSource='ds'}," + + " errorResponse={error=druidException, errorCode=QueryNotSupported, persona=USER, category=UNCATEGORIZED, errorMessage=QueryNotSupported, context={}}}", + SQL_STATEMENT_RESULT.toString() + ); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index de065b65e63b..cad182aef5d6 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -134,7 +134,6 @@ import org.apache.druid.server.coordination.DataSegmentAnnouncer; import org.apache.druid.server.coordination.NoopDataSegmentAnnouncer; import org.apache.druid.server.security.AuthConfig; -import org.apache.druid.server.security.AuthTestUtils; import org.apache.druid.sql.DirectStatement; import org.apache.druid.sql.SqlQueryPlus; import org.apache.druid.sql.SqlStatementFactory; @@ -494,7 +493,7 @@ public String getFormatString() qf.operatorTable(), qf.macroTable(), PLANNER_CONFIG_DEFAULT, - AuthTestUtils.TEST_AUTHORIZER_MAPPER, + CalciteTests.TEST_AUTHORIZER_MAPPER, objectMapper, CalciteTests.DRUID_SCHEMA_NAME, new CalciteRulesManager(ImmutableSet.of()), @@ -953,10 +952,12 @@ protected void verifyCounters(CounterSnapshotsTree counterSnapshotsTree) worker, channel ); - Assert.assertTrue(StringUtils.format("Counters not found for stage [%d], worker [%d], channel [%s]", - stage, - worker, - channel), channelToCounters.containsKey(channel)); + Assert.assertTrue(StringUtils.format( + "Counters not found for stage [%d], worker [%d], channel [%s]", + stage, + worker, + channel + ), channelToCounters.containsKey(channel)); counter.matchQuerySnapshot(errorMessageFormat, channelToCounters.get(channel)); } ); From 5ab7771faba080535b5e4ad422078e93fcd176a5 Mon Sep 17 00:00:00 2001 From: cryptoe Date: Tue, 27 Jun 2023 23:49:37 +0530 Subject: [PATCH 11/15] Removing exception details from QueryException --- .../sql/SqlMsqStatementResourcePostTest.java | 64 +++++++++++++++++-- .../sql/entity/ResultSetInformationTest.java | 2 +- .../org/apache/druid/error/ErrorResponse.java | 3 +- .../apache/druid/query/BadQueryException.java | 2 +- .../query/QueryCapacityExceededException.java | 6 +- .../apache/druid/query/QueryException.java | 35 +++------- .../query/QueryInterruptedException.java | 2 +- .../druid/query/QueryTimeoutException.java | 8 +-- .../query/QueryUnsupportedException.java | 4 +- .../druid/query/QueryExceptionTest.java | 6 +- .../apache/druid/server/QueryResource.java | 3 +- .../druid/client/JsonParserIteratorTest.java | 6 +- .../druid/sql/calcite/rel/Windowing.java | 1 - .../druid/sql/avatica/ErrorHandlerTest.java | 2 +- 14 files changed, 88 insertions(+), 56 deletions(-) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java index 7729e306deed..4408abdeb30f 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.calcite.sql.type.SqlTypeName; @@ -47,12 +46,13 @@ import javax.ws.rs.core.Response; import java.io.IOException; +import java.util.HashMap; import java.util.List; +import java.util.Map; public class SqlMsqStatementResourcePostTest extends MSQTestBase { private SqlStatementResource resource; - public static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); @Before public void init() @@ -131,6 +131,30 @@ public void testMSQSelectQueryTest() throws IOException } + @Test + public void nonSupportedModes() + { + for (ImmutableMap context : ImmutableList.of(ImmutableMap.of( + QueryContexts.CTX_EXECUTION_MODE, + ExecutionMode.SYNC.name() + ), ImmutableMap.of())) { + SqlStatementResourceTest.assertExceptionMessage( + resource.doPost(new SqlQuery( + "select * from foo", + null, + false, + false, + false, + (Map) context, + null + ), SqlStatementResourceTest.makeOkRequest()), + "The statement sql api only supports sync mode[ASYNC]. Please set context parameter [executionMode=ASYNC] in the context payload", + Response.Status.BAD_REQUEST + ); + } + } + + @Test public void insertCannotBeEmptyFaultTest() { @@ -174,9 +198,34 @@ protected DruidException makeException(DruidException.DruidExceptionBuilder bob) Assert.assertEquals(expected, actual); } - private static ImmutableMap defaultAsyncContext() + @Test + public void testExplain() throws IOException { - return ImmutableMap.of(QueryContexts.CTX_EXECUTION_MODE, ExecutionMode.ASYNC.name()); + Map context = defaultAsyncContext(); + context.put("sqlQueryId", "queryId"); + Response response = resource.doPost(new SqlQuery( + "explain plan for select * from foo", + null, + false, + false, + false, + context, + null + ), SqlStatementResourceTest.makeOkRequest()); + + Assert.assertEquals( + "{PLAN=[{\"query\":" + + "{\"queryType\":\"scan\"," + + "\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"}," + + "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," + + "\"resultFormat\":\"compactedList\"," + + "\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"]," + + "\"legacy\":false," + + "\"context\":{\"executionMode\":\"ASYNC\",\"scanSignature\":\"[{\\\"name\\\":\\\"__time\\\",\\\"type\\\":\\\"LONG\\\"},{\\\"name\\\":\\\"cnt\\\",\\\"type\\\":\\\"LONG\\\"},{\\\"name\\\":\\\"dim1\\\",\\\"type\\\":\\\"STRING\\\"},{\\\"name\\\":\\\"dim2\\\",\\\"type\\\":\\\"STRING\\\"},{\\\"name\\\":\\\"dim3\\\",\\\"type\\\":\\\"STRING\\\"},{\\\"name\\\":\\\"m1\\\",\\\"type\\\":\\\"FLOAT\\\"},{\\\"name\\\":\\\"m2\\\",\\\"type\\\":\\\"DOUBLE\\\"},{\\\"name\\\":\\\"unique_dim1\\\",\\\"type\\\":\\\"COMPLEX\\\"}]\",\"sqlQueryId\":\"queryId\"},\"granularity\":{\"type\":\"all\"}},\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX\"}],\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"},{\"queryColumn\":\"dim2\",\"outputColumn\":\"dim2\"},{\"queryColumn\":\"dim3\",\"outputColumn\":\"dim3\"},{\"queryColumn\":\"cnt\",\"outputColumn\":\"cnt\"},{\"queryColumn\":\"m1\",\"outputColumn\":\"m1\"},{\"queryColumn\":\"m2\",\"outputColumn\":\"m2\"},{\"queryColumn\":\"unique_dim1\",\"outputColumn\":\"unique_dim1\"}]}]," + + " RESOURCES=[{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]," + + " ATTRIBUTES={\"statementType\":\"SELECT\"}}", + String.valueOf(SqlStatementResourceTest.getResultRowsFromResponse(response).get(0)) + ); } @Test @@ -197,4 +246,11 @@ public void forbiddenTest() } + private static Map defaultAsyncContext() + { + Map context = new HashMap(); + context.put(QueryContexts.CTX_EXECUTION_MODE, ExecutionMode.ASYNC.name()); + return context; + } + } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/ResultSetInformationTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/ResultSetInformationTest.java index 0a186c646c0b..14d04b1b76d5 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/ResultSetInformationTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/entity/ResultSetInformationTest.java @@ -31,7 +31,7 @@ public class ResultSetInformationTest { public static final ObjectMapper MAPPER = new ObjectMapper(); - public static final ResultSetInformation RESULTS = new ResultSetInformation(ResultFormat.OBJECT, 1l, 1l, "ds", + public static final ResultSetInformation RESULTS = new ResultSetInformation(ResultFormat.OBJECT, 1L, 1L, "ds", ImmutableList.of( ImmutableList.of("1"), ImmutableList.of("2"), diff --git a/processing/src/main/java/org/apache/druid/error/ErrorResponse.java b/processing/src/main/java/org/apache/druid/error/ErrorResponse.java index 2a89dc662687..7b571cca2719 100644 --- a/processing/src/main/java/org/apache/druid/error/ErrorResponse.java +++ b/processing/src/main/java/org/apache/druid/error/ErrorResponse.java @@ -79,8 +79,7 @@ public static ErrorResponse fromMap(Map map) nullOrString(map.get("error")), nullOrString(map.get("errorMessage")), nullOrString(map.get("errorClass")), - nullOrString(map.get("host")), - null + nullOrString(map.get("host")) ) ); } else { diff --git a/processing/src/main/java/org/apache/druid/query/BadQueryException.java b/processing/src/main/java/org/apache/druid/query/BadQueryException.java index 0689ffc3a11d..dc498e85e91f 100644 --- a/processing/src/main/java/org/apache/druid/query/BadQueryException.java +++ b/processing/src/main/java/org/apache/druid/query/BadQueryException.java @@ -38,6 +38,6 @@ protected BadQueryException(String errorCode, String errorMessage, String errorC protected BadQueryException(Throwable cause, String errorCode, String errorMessage, String errorClass, String host) { - super(cause, errorCode, errorMessage, errorClass, host, null); + super(cause, errorCode, errorMessage, errorClass, host); } } diff --git a/processing/src/main/java/org/apache/druid/query/QueryCapacityExceededException.java b/processing/src/main/java/org/apache/druid/query/QueryCapacityExceededException.java index 2229e1fe9135..694fbb780cb6 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryCapacityExceededException.java +++ b/processing/src/main/java/org/apache/druid/query/QueryCapacityExceededException.java @@ -47,12 +47,12 @@ public class QueryCapacityExceededException extends QueryException public QueryCapacityExceededException(int capacity) { - super(QUERY_CAPACITY_EXCEEDED_ERROR_CODE, makeTotalErrorMessage(capacity), ERROR_CLASS, null, null); + super(QUERY_CAPACITY_EXCEEDED_ERROR_CODE, makeTotalErrorMessage(capacity), ERROR_CLASS, null); } public QueryCapacityExceededException(String lane, int capacity) { - super(QUERY_CAPACITY_EXCEEDED_ERROR_CODE, makeLaneErrorMessage(lane, capacity), ERROR_CLASS, null, null); + super(QUERY_CAPACITY_EXCEEDED_ERROR_CODE, makeLaneErrorMessage(lane, capacity), ERROR_CLASS, null); } /** @@ -77,7 +77,7 @@ public QueryCapacityExceededException( @JsonProperty("host") String host ) { - super(errorCode, errorMessage, errorClass, host, null); + super(errorCode, errorMessage, errorClass, host); } @VisibleForTesting diff --git a/processing/src/main/java/org/apache/druid/query/QueryException.java b/processing/src/main/java/org/apache/druid/query/QueryException.java index 75634e457637..def86471bcd4 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryException.java +++ b/processing/src/main/java/org/apache/druid/query/QueryException.java @@ -20,16 +20,13 @@ package org.apache.druid.query; import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Joiner; import org.apache.druid.common.exception.SanitizableException; import org.apache.druid.java.util.common.StringUtils; import javax.annotation.Nullable; import javax.validation.constraints.NotNull; import java.net.InetAddress; -import java.util.Map; import java.util.Objects; import java.util.function.Function; @@ -155,11 +152,9 @@ public static FailType fromErrorCode(String errorCode) private final String errorClass; private final String host; - private final Map details; - protected QueryException(Throwable cause, String errorCode, String errorClass, String host) { - this(cause, errorCode, cause == null ? null : cause.getMessage(), errorClass, host, null); + this(cause, errorCode, cause == null ? null : cause.getMessage(), errorClass, host); } protected QueryException( @@ -167,15 +162,13 @@ protected QueryException( String errorCode, String errorMessage, String errorClass, - String host, - Map details + String host ) { super(errorMessage, cause); this.errorCode = errorCode; this.errorClass = errorClass; this.host = host; - this.details = details; } @JsonCreator @@ -183,15 +176,13 @@ public QueryException( @JsonProperty("error") @Nullable String errorCode, @JsonProperty("errorMessage") String errorMessage, @JsonProperty("errorClass") @Nullable String errorClass, - @JsonProperty("host") @Nullable String host, - @JsonProperty("details") @Nullable Map details + @JsonProperty("host") @Nullable String host ) { super(errorMessage); this.errorCode = errorCode; this.errorClass = errorClass; this.host = host; - this.details = details; } @Nullable @@ -220,14 +211,6 @@ public String getHost() return host; } - @JsonProperty - @Nullable - @JsonInclude(JsonInclude.Include.NON_NULL) - public Map getDetails() - { - return details; - } - @Nullable protected static String resolveHostname() { @@ -242,7 +225,7 @@ protected static String resolveHostname() @Override public QueryException sanitize(@NotNull Function errorMessageTransformFunction) { - return new QueryException(errorCode, errorMessageTransformFunction.apply(getMessage()), null, null, null); + return new QueryException(errorCode, errorMessageTransformFunction.apply(getMessage()), null, null); } public FailType getFailType() @@ -254,13 +237,12 @@ public FailType getFailType() public String toString() { return StringUtils.format( - "%s{msg=%s, code=%s, class=%s, host=%s, details=%s}", + "%s{msg=%s, code=%s, class=%s, host=%s}", getClass().getSimpleName(), getMessage(), getErrorCode(), getErrorClass(), - getHost(), - getDetails() != null ? Joiner.on(",").withKeyValueSeparator("=").join(getDetails()) : null + getHost() ); } @@ -276,13 +258,12 @@ public boolean equals(Object o) QueryException that = (QueryException) o; return Objects.equals(errorCode, that.errorCode) && Objects.equals(errorClass, that.errorClass) - && Objects.equals(host, that.host) - && Objects.equals(details, that.details); + && Objects.equals(host, that.host); } @Override public int hashCode() { - return Objects.hash(errorCode, errorClass, host, details); + return Objects.hash(errorCode, errorClass, host); } } diff --git a/processing/src/main/java/org/apache/druid/query/QueryInterruptedException.java b/processing/src/main/java/org/apache/druid/query/QueryInterruptedException.java index 8c93bc13b75a..f1893cf3eaaa 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryInterruptedException.java +++ b/processing/src/main/java/org/apache/druid/query/QueryInterruptedException.java @@ -50,7 +50,7 @@ public QueryInterruptedException( @JsonProperty("host") @Nullable String host ) { - super(errorCode, errorMessage, errorClass, host, null); + super(errorCode, errorMessage, errorClass, host); } /** diff --git a/processing/src/main/java/org/apache/druid/query/QueryTimeoutException.java b/processing/src/main/java/org/apache/druid/query/QueryTimeoutException.java index af1290be4f55..84be0a5ca37a 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryTimeoutException.java +++ b/processing/src/main/java/org/apache/druid/query/QueryTimeoutException.java @@ -48,21 +48,21 @@ public QueryTimeoutException( @JsonProperty("host") @Nullable String host ) { - super(errorCode, errorMessage, errorClass, host, null); + super(errorCode, errorMessage, errorClass, host); } public QueryTimeoutException() { - super(QUERY_TIMEOUT_ERROR_CODE, ERROR_MESSAGE, ERROR_CLASS, resolveHostname(), null); + super(QUERY_TIMEOUT_ERROR_CODE, ERROR_MESSAGE, ERROR_CLASS, resolveHostname()); } public QueryTimeoutException(String errorMessage) { - super(QUERY_TIMEOUT_ERROR_CODE, errorMessage, ERROR_CLASS, resolveHostname(), null); + super(QUERY_TIMEOUT_ERROR_CODE, errorMessage, ERROR_CLASS, resolveHostname()); } public QueryTimeoutException(String errorMessage, String host) { - super(QUERY_TIMEOUT_ERROR_CODE, errorMessage, ERROR_CLASS, host, null); + super(QUERY_TIMEOUT_ERROR_CODE, errorMessage, ERROR_CLASS, host); } } diff --git a/processing/src/main/java/org/apache/druid/query/QueryUnsupportedException.java b/processing/src/main/java/org/apache/druid/query/QueryUnsupportedException.java index 81d69ea4849a..81d82a94871a 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryUnsupportedException.java +++ b/processing/src/main/java/org/apache/druid/query/QueryUnsupportedException.java @@ -46,11 +46,11 @@ public QueryUnsupportedException( @JsonProperty("host") @Nullable String host ) { - super(errorCode, errorMessage, errorClass, host, null); + super(errorCode, errorMessage, errorClass, host); } public QueryUnsupportedException(String errorMessage) { - super(QUERY_UNSUPPORTED_ERROR_CODE, errorMessage, ERROR_CLASS, resolveHostname(), null); + super(QUERY_UNSUPPORTED_ERROR_CODE, errorMessage, ERROR_CLASS, resolveHostname()); } } diff --git a/processing/src/test/java/org/apache/druid/query/QueryExceptionTest.java b/processing/src/test/java/org/apache/druid/query/QueryExceptionTest.java index c49feb532e7e..10a0b78521da 100644 --- a/processing/src/test/java/org/apache/druid/query/QueryExceptionTest.java +++ b/processing/src/test/java/org/apache/druid/query/QueryExceptionTest.java @@ -36,7 +36,7 @@ public class QueryExceptionTest @Test public void testSanitizeWithTransformFunctionReturningNull() { - QueryException queryException = new QueryException(ERROR_CODE, ERROR_MESSAGE_ORIGINAL, ERROR_CLASS, HOST, null); + QueryException queryException = new QueryException(ERROR_CODE, ERROR_MESSAGE_ORIGINAL, ERROR_CLASS, HOST); AtomicLong callCount = new AtomicLong(0); QueryException actual = queryException.sanitize(s -> { @@ -56,7 +56,7 @@ public void testSanitizeWithTransformFunctionReturningNull() @Test public void testSanitizeWithTransformFunctionReturningNewString() { - QueryException queryException = new QueryException(ERROR_CODE, ERROR_MESSAGE_ORIGINAL, ERROR_CLASS, HOST, null); + QueryException queryException = new QueryException(ERROR_CODE, ERROR_MESSAGE_ORIGINAL, ERROR_CLASS, HOST); AtomicLong callCount = new AtomicLong(0); QueryException actual = queryException.sanitize(s -> { @@ -116,7 +116,7 @@ public void testCanConstructWithoutThrowable() @Test public void testToStringReturnsUsefulInformation() { - QueryException queryException = new QueryException(ERROR_CODE, ERROR_MESSAGE_ORIGINAL, ERROR_CLASS, HOST, null); + QueryException queryException = new QueryException(ERROR_CODE, ERROR_MESSAGE_ORIGINAL, ERROR_CLASS, HOST); String exceptionToString = queryException.toString(); Assert.assertTrue(exceptionToString.startsWith(QueryException.class.getSimpleName())); Assert.assertTrue(exceptionToString.contains("msg=" + ERROR_MESSAGE_ORIGINAL)); diff --git a/server/src/main/java/org/apache/druid/server/QueryResource.java b/server/src/main/java/org/apache/druid/server/QueryResource.java index 58fd1899c018..2db205ca0bed 100644 --- a/server/src/main/java/org/apache/druid/server/QueryResource.java +++ b/server/src/main/java/org/apache/druid/server/QueryResource.java @@ -238,8 +238,7 @@ public Response doPost( QueryException.UNKNOWN_EXCEPTION_ERROR_CODE, "Unhandled exception made it to the top", e.getClass().getName(), - req.getRemoteHost(), - null + req.getRemoteHost() ); out.write(jsonMapper.writeValueAsBytes(responseException)); } diff --git a/server/src/test/java/org/apache/druid/client/JsonParserIteratorTest.java b/server/src/test/java/org/apache/druid/client/JsonParserIteratorTest.java index 1f0683f0456e..80cf0f6d045b 100644 --- a/server/src/test/java/org/apache/druid/client/JsonParserIteratorTest.java +++ b/server/src/test/java/org/apache/druid/client/JsonParserIteratorTest.java @@ -78,8 +78,7 @@ public void testConvertFutureTimeoutToQueryTimeoutException() QueryException.QUERY_TIMEOUT_ERROR_CODE, "timeout exception conversion test", null, - HOST, - null + HOST ) ), URL, @@ -206,7 +205,6 @@ public void testConvertQueryExceptionWithNullErrorCodeToQueryInterruptedExceptio null, "query exception test", null, - null, null ))), URL, @@ -226,7 +224,7 @@ public void testConvertQueryExceptionWithNonNullErrorCodeToQueryInterruptedExcep JsonParserIterator iterator = new JsonParserIterator<>( JAVA_TYPE, Futures.immediateFuture( - mockErrorResponse(new QueryException("test error", "query exception test", null, null, null)) + mockErrorResponse(new QueryException("test error", "query exception test", null, null)) ), URL, null, diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java index 4e7313c08949..af5983d6edbf 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java @@ -323,7 +323,6 @@ public ArrayList getOrdering() QueryException.SQL_QUERY_UNSUPPORTED_ERROR_CODE, StringUtils.format("Cannot handle ordering with direction[%s]", field.direction), null, - null, null ); } diff --git a/sql/src/test/java/org/apache/druid/sql/avatica/ErrorHandlerTest.java b/sql/src/test/java/org/apache/druid/sql/avatica/ErrorHandlerTest.java index 158d17a441bc..2efa65d1a7b5 100644 --- a/sql/src/test/java/org/apache/druid/sql/avatica/ErrorHandlerTest.java +++ b/sql/src/test/java/org/apache/druid/sql/avatica/ErrorHandlerTest.java @@ -41,7 +41,7 @@ public void testErrorHandlerSanitizesErrorAsExpected() Mockito.when(serverConfig.getErrorResponseTransformStrategy()) .thenReturn(emptyAllowedRegexErrorResponseTransformStrategy); ErrorHandler errorHandler = new ErrorHandler(serverConfig); - QueryException input = new QueryException("error", "error message", "error class", "host", null); + QueryException input = new QueryException("error", "error message", "error class", "host"); RuntimeException output = errorHandler.sanitize(input); Assert.assertNull(output.getMessage()); From 8055386b31904733be93c09a2f86b418c9f54676 Mon Sep 17 00:00:00 2001 From: cryptoe Date: Wed, 28 Jun 2023 10:07:22 +0530 Subject: [PATCH 12/15] Adding more tests for forbidden exception. Code coverage things. Review comments. --- .../sql/resources/SqlStatementResource.java | 103 ++++++++++++------ .../msq/sql/SqlStatementResourceTest.java | 18 +++ .../apache/druid/msq/test/MSQTestBase.java | 2 +- .../org/apache/druid/query/ExecutionMode.java | 7 ++ .../apache/druid/query/QueryException.java | 30 +---- .../apache/druid/query/QueryContextsTest.java | 22 ++++ .../druid/client/JsonParserIteratorTest.java | 7 +- .../rpc/indexing/OverlordClientImplTest.java | 58 ++++++++++ .../druid/sql/calcite/util/CalciteTests.java | 34 ++++++ 9 files changed, 212 insertions(+), 69 deletions(-) create mode 100644 server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java index 029d4fd31ed2..d8453afa5f61 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java @@ -222,7 +222,14 @@ public Response doGetStatus( ) { try { - AuthorizationUtils.authorizeAllResourceActions(req, Collections.emptyList(), authorizerMapper); + Access authResult = AuthorizationUtils.authorizeAllResourceActions( + req, + Collections.emptyList(), + authorizerMapper + ); + if (!authResult.isAllowed()) { + throw new ForbiddenException(authResult.toString()); + } final AuthenticationResult authenticationResult = AuthorizationUtils.authenticationResultFromRequest(req); Optional sqlStatementResult = getStatementStatus( @@ -230,6 +237,7 @@ public Response doGetStatus( authenticationResult.getIdentity(), true ); + if (sqlStatementResult.isPresent()) { return Response.ok().entity(sqlStatementResult.get()).build(); } else { @@ -265,31 +273,41 @@ public Response doGetResults( @Context final HttpServletRequest req ) { - if (offset != null && offset < 0) { - return buildNonOkResponse( - DruidException.forPersona(DruidException.Persona.USER) - .ofCategory(DruidException.Category.INVALID_INPUT) - .build( - "offset cannot be negative. Please pass a positive number." - ) - ); - } - if (numberOfRows != null && numberOfRows < 0) { - return buildNonOkResponse( - DruidException.forPersona(DruidException.Persona.USER) - .ofCategory(DruidException.Category.INVALID_INPUT) - .build( - "numRows cannot be negative. Please pass a positive number." - ) + try { + Access authResult = AuthorizationUtils.authorizeAllResourceActions( + req, + Collections.emptyList(), + authorizerMapper ); - } + if (!authResult.isAllowed()) { + throw new ForbiddenException(authResult.toString()); + } + final AuthenticationResult authenticationResult = AuthorizationUtils.authenticationResultFromRequest(req); + + if (offset != null && offset < 0) { + return buildNonOkResponse( + DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build( + "offset cannot be negative. Please pass a positive number." + ) + ); + } + if (numberOfRows != null && numberOfRows < 0) { + return buildNonOkResponse( + DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build( + "numRows cannot be negative. Please pass a positive number." + ) + ); + } - final long start = offset == null ? 0 : offset; - final long last = SqlStatementResourceHelper.getLastIndex(numberOfRows, start); + final long start = offset == null ? 0 : offset; + final long last = SqlStatementResourceHelper.getLastIndex(numberOfRows, start); - try { - AuthorizationUtils.authorizeAllResourceActions(req, Collections.emptyList(), authorizerMapper); - final AuthenticationResult authenticationResult = AuthorizationUtils.authenticationResultFromRequest(req); + + getStatementStatus(queryId, authenticationResult.getIdentity(), false); TaskStatusResponse taskResponse = contactOverlord(overlordClient.taskStatus(queryId)); if (taskResponse == null) { @@ -300,6 +318,8 @@ public Response doGetResults( if (statusPlus == null || !MSQControllerTask.TYPE.equals(statusPlus.getType())) { return Response.status(Response.Status.NOT_FOUND).build(); } + + MSQControllerTask msqControllerTask = getMSQControllerTaskOrThrow(queryId, authenticationResult.getIdentity()); SqlStatementState sqlStatementState = SqlStatementResourceHelper.getSqlStatementState(statusPlus); if (sqlStatementState == SqlStatementState.RUNNING || sqlStatementState == SqlStatementState.ACCEPTED) { @@ -322,7 +342,6 @@ public Response doGetResults( ) ); } else { - MSQControllerTask msqControllerTask = getMSQControllerTaskOrThrow(queryId, authenticationResult.getIdentity()); Optional> signature = SqlStatementResourceHelper.getSignature(msqControllerTask); if (!signature.isPresent()) { return Response.ok().build(); @@ -392,7 +411,14 @@ public Response deleteQuery(@PathParam("id") final String queryId, @Context fina { try { - AuthorizationUtils.authorizeAllResourceActions(req, Collections.emptyList(), authorizerMapper); + Access authResult = AuthorizationUtils.authorizeAllResourceActions( + req, + Collections.emptyList(), + authorizerMapper + ); + if (!authResult.isAllowed()) { + throw new ForbiddenException(authResult.toString()); + } final AuthenticationResult authenticationResult = AuthorizationUtils.authenticationResultFromRequest(req); Optional sqlStatementResult = getStatementStatus( @@ -504,7 +530,14 @@ private Response buildTaskResponse(Sequence sequence, String user) if (numRows != 1) { throw new RE("Expected a single row but got [%d] rows. Please check broker logs for more information.", numRows); } - String taskId = (String) rows.get(0)[0]; + Object[] firstRow = rows.get(0); + if (firstRow == null || firstRow.length != 1) { + throw new RE( + "Expected a single column but got [%s] columns. Please check broker logs for more information.", + firstRow == null ? 0 : firstRow.length + ); + } + String taskId = String.valueOf(firstRow[0]); Optional statementResult = getStatementStatus(taskId, user, true); @@ -611,13 +644,17 @@ private MSQControllerTask getMSQControllerTaskOrThrow(String queryId, String cur SqlStatementResourceHelper.isMSQPayload(taskPayloadResponse, queryId); MSQControllerTask msqControllerTask = (MSQControllerTask) taskPayloadResponse.getPayload(); - if (currentUser == null || !currentUser.equals(msqControllerTask.getQuerySpec() - .getQuery() - .getContext() - .get(MSQTaskQueryMaker.USER_KEY))) { - throw DruidException.forPersona(DruidException.Persona.USER) - .ofCategory(DruidException.Category.FORBIDDEN) - .build(Access.DEFAULT_ERROR_MESSAGE); + String queryUser = String.valueOf(msqControllerTask.getQuerySpec() + .getQuery() + .getContext() + .get(MSQTaskQueryMaker.USER_KEY)); + if (currentUser == null || !currentUser.equals(queryUser)) { + throw new ForbiddenException(StringUtils.format( + "The current user[%s] cannot view query id[%s] since the query is owned by user[%s]", + currentUser, + queryId, + queryUser + )); } return msqControllerTask; } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java index 4002d3614528..3f2e7288b624 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlStatementResourceTest.java @@ -921,6 +921,24 @@ public void testMSQInsertRunningQuery() ); } + @Test + public void forbiddenTests() + { + + Assert.assertEquals(Response.Status.FORBIDDEN.getStatusCode(), + resource.doGetStatus(RUNNING_SELECT_MSQ_QUERY, + makeExpectedReq(CalciteTests.SUPER_USER_AUTH_RESULT)).getStatus()); + + Assert.assertEquals(Response.Status.FORBIDDEN.getStatusCode(), + resource.doGetResults(RUNNING_SELECT_MSQ_QUERY, + null, + null, + makeExpectedReq(CalciteTests.SUPER_USER_AUTH_RESULT)).getStatus()); + Assert.assertEquals(Response.Status.FORBIDDEN.getStatusCode(), + resource.deleteQuery(RUNNING_SELECT_MSQ_QUERY, + makeExpectedReq(CalciteTests.SUPER_USER_AUTH_RESULT)).getStatus()); + } + @Test public void testIsEnabled() { diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index cad182aef5d6..610d2a4b83c1 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -493,7 +493,7 @@ public String getFormatString() qf.operatorTable(), qf.macroTable(), PLANNER_CONFIG_DEFAULT, - CalciteTests.TEST_AUTHORIZER_MAPPER, + CalciteTests.TEST_EXTERNAL_AUTHORIZER_MAPPER, objectMapper, CalciteTests.DRUID_SCHEMA_NAME, new CalciteRulesManager(ImmutableSet.of()), diff --git a/processing/src/main/java/org/apache/druid/query/ExecutionMode.java b/processing/src/main/java/org/apache/druid/query/ExecutionMode.java index f17498b29929..9809d66051bf 100644 --- a/processing/src/main/java/org/apache/druid/query/ExecutionMode.java +++ b/processing/src/main/java/org/apache/druid/query/ExecutionMode.java @@ -22,7 +22,14 @@ public enum ExecutionMode { + /** + * This mode executes the query in a blocking way. The results are returned as part of the original post query call. Current sql/native endpoints are sync execution. + */ SYNC, + + /** + * This mode executes the query in a non-blocking way. The results are returned as part of subsequent get results call. Currently, the msq engine uses this mode of execution. + */ ASYNC } diff --git a/processing/src/main/java/org/apache/druid/query/QueryException.java b/processing/src/main/java/org/apache/druid/query/QueryException.java index def86471bcd4..5dbdef6ddb7e 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryException.java +++ b/processing/src/main/java/org/apache/druid/query/QueryException.java @@ -27,7 +27,6 @@ import javax.annotation.Nullable; import javax.validation.constraints.NotNull; import java.net.InetAddress; -import java.util.Objects; import java.util.function.Function; /** @@ -157,13 +156,7 @@ protected QueryException(Throwable cause, String errorCode, String errorClass, S this(cause, errorCode, cause == null ? null : cause.getMessage(), errorClass, host); } - protected QueryException( - Throwable cause, - String errorCode, - String errorMessage, - String errorClass, - String host - ) + protected QueryException(Throwable cause, String errorCode, String errorMessage, String errorClass, String host) { super(errorMessage, cause); this.errorCode = errorCode; @@ -245,25 +238,4 @@ public String toString() getHost() ); } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - QueryException that = (QueryException) o; - return Objects.equals(errorCode, that.errorCode) - && Objects.equals(errorClass, that.errorClass) - && Objects.equals(host, that.host); - } - - @Override - public int hashCode() - { - return Objects.hash(errorCode, errorClass, host); - } } diff --git a/processing/src/test/java/org/apache/druid/query/QueryContextsTest.java b/processing/src/test/java/org/apache/druid/query/QueryContextsTest.java index d47bb558fe97..38b5384ded9d 100644 --- a/processing/src/test/java/org/apache/druid/query/QueryContextsTest.java +++ b/processing/src/test/java/org/apache/druid/query/QueryContextsTest.java @@ -295,4 +295,26 @@ public void testGetEnum() () -> query.context().getEnum("e2", QueryContexts.Vectorize.class, QueryContexts.Vectorize.FALSE) ); } + + @Test + public void testExecutionModeEnum() + { + Query query = new TestQuery( + new TableDataSource("test"), + new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("0/100"))), + false, + ImmutableMap.of(QueryContexts.CTX_EXECUTION_MODE, "SYNC", QueryContexts.CTX_EXECUTION_MODE + "_1", "ASYNC") + ); + + Assert.assertEquals( + ExecutionMode.SYNC, + query.context().getEnum(QueryContexts.CTX_EXECUTION_MODE, ExecutionMode.class, ExecutionMode.ASYNC) + ); + + Assert.assertEquals( + ExecutionMode.ASYNC, + query.context().getEnum(QueryContexts.CTX_EXECUTION_MODE + "_1", ExecutionMode.class, ExecutionMode.SYNC) + ); + } + } diff --git a/server/src/test/java/org/apache/druid/client/JsonParserIteratorTest.java b/server/src/test/java/org/apache/druid/client/JsonParserIteratorTest.java index 80cf0f6d045b..f896067646cc 100644 --- a/server/src/test/java/org/apache/druid/client/JsonParserIteratorTest.java +++ b/server/src/test/java/org/apache/druid/client/JsonParserIteratorTest.java @@ -201,12 +201,7 @@ public void testConvertQueryExceptionWithNullErrorCodeToQueryInterruptedExceptio { JsonParserIterator iterator = new JsonParserIterator<>( JAVA_TYPE, - Futures.immediateFuture(mockErrorResponse(new QueryException( - null, - "query exception test", - null, - null - ))), + Futures.immediateFuture(mockErrorResponse(new QueryException(null, "query exception test", null, null))), URL, null, HOST, diff --git a/server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java b/server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java new file mode 100644 index 000000000000..26fb972ac5cd --- /dev/null +++ b/server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java @@ -0,0 +1,58 @@ +/* + * 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.rpc.indexing; + +import com.fasterxml.jackson.core.JsonProcessingException; +import org.apache.druid.client.indexing.ClientKillUnusedSegmentsTaskQuery; +import org.apache.druid.client.indexing.ClientTaskQuery; +import org.apache.druid.client.indexing.TaskPayloadResponse; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.rpc.MockServiceClient; +import org.apache.druid.rpc.RequestBuilder; +import org.jboss.netty.handler.codec.http.HttpMethod; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Collections; +import java.util.concurrent.ExecutionException; + +public class OverlordClientImplTest +{ + + + @Test + public void testTaskPayload() throws ExecutionException, InterruptedException, JsonProcessingException + { + final String taskID = "taskId_1"; + MockServiceClient client = new MockServiceClient(); + final OverlordClientImpl overlordClient = new OverlordClientImpl(client, DefaultObjectMapper.INSTANCE); + + ClientTaskQuery clientTaskQuery = new ClientKillUnusedSegmentsTaskQuery(taskID, "test", null, null); + + client.expect(new RequestBuilder(HttpMethod.GET, "/druid/indexer/v1/task/" + taskID), + HttpResponseStatus.OK, + Collections.emptyMap(), + DefaultObjectMapper.INSTANCE.writeValueAsBytes(new TaskPayloadResponse(taskID, clientTaskQuery)) + ); + + Assert.assertEquals(clientTaskQuery, overlordClient.taskPayload(taskID).get().getPayload()); + } +} diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java index 8bffc909efa2..0dfec3759e0c 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java @@ -135,6 +135,40 @@ public Authorizer getAuthorizer(String name) }; } }; + + public static final AuthorizerMapper TEST_EXTERNAL_AUTHORIZER_MAPPER = new AuthorizerMapper(null) + { + @Override + public Authorizer getAuthorizer(String name) + { + return (authenticationResult, resource, action) -> { + if (TEST_SUPERUSER_NAME.equals(authenticationResult.getIdentity())) { + return Access.OK; + } + + switch (resource.getType()) { + case ResourceType.DATASOURCE: + if (FORBIDDEN_DATASOURCE.equals(resource.getName())) { + return new Access(false); + } else { + return Access.OK; + } + case ResourceType.VIEW: + if ("forbiddenView".equals(resource.getName())) { + return new Access(false); + } else { + return Access.OK; + } + case ResourceType.QUERY_CONTEXT: + case ResourceType.EXTERNAL: + return Access.OK; + default: + return new Access(false); + } + }; + } + }; + public static final AuthenticatorMapper TEST_AUTHENTICATOR_MAPPER; static { From 164337dd39f536ade6fd88fbf1a984f4837cdc01 Mon Sep 17 00:00:00 2001 From: cryptoe Date: Wed, 28 Jun 2023 10:16:14 +0530 Subject: [PATCH 13/15] Spotbugs. --- .../org/apache/druid/msq/sql/entity/SqlStatementResult.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/SqlStatementResult.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/SqlStatementResult.java index a0bfc7f2694b..de66550a5875 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/SqlStatementResult.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/entity/SqlStatementResult.java @@ -149,7 +149,7 @@ public boolean equals(Object o) that.durationMs ) && Objects.equals(resultSetInformation, that.resultSetInformation) && Objects.equals( errorResponse == null ? null : errorResponse.getAsMap(), - that.errorResponse == null ? null : errorResponse.getAsMap() + that.errorResponse == null ? null : that.errorResponse.getAsMap() ); } From 56ceec145e794a0220612370b15348a98642c2d1 Mon Sep 17 00:00:00 2001 From: cryptoe Date: Wed, 28 Jun 2023 11:43:42 +0530 Subject: [PATCH 14/15] Error Message test case fix. --- .../apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java index 4408abdeb30f..ceae64dcf703 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/SqlMsqStatementResourcePostTest.java @@ -189,7 +189,7 @@ protected DruidException makeException(DruidException.DruidExceptionBuilder bob) { DruidException e = bob.forPersona(DruidException.Persona.USER) .ofCategory(DruidException.Category.UNCATEGORIZED) - .build(insertCannotBeEmpty.getMessage()); + .build(insertCannotBeEmpty.getFault().getErrorMessage()); e.withContext("dataSource", insertCannotBeEmptyFault.getDataSource()); return e; } From 15778a80bd275f495d08a369dce6a9fc69ad868c Mon Sep 17 00:00:00 2001 From: cryptoe Date: Wed, 28 Jun 2023 14:14:26 +0530 Subject: [PATCH 15/15] Removing extra status call. --- .../apache/druid/msq/sql/resources/SqlStatementResource.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java index d8453afa5f61..6ce5c7800525 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/resources/SqlStatementResource.java @@ -306,9 +306,6 @@ public Response doGetResults( final long start = offset == null ? 0 : offset; final long last = SqlStatementResourceHelper.getLastIndex(numberOfRows, start); - - getStatementStatus(queryId, authenticationResult.getIdentity(), false); - TaskStatusResponse taskResponse = contactOverlord(overlordClient.taskStatus(queryId)); if (taskResponse == null) { return Response.status(Response.Status.NOT_FOUND).build();