From a27e2399a7e691d133283a751274dbc7442c1719 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Tue, 24 Sep 2024 02:11:23 -0700 Subject: [PATCH] MSQ profile for Brokers and Historicals. This patch adds a profile of MSQ named "Dart" that runs on Brokers and Historicals, and which is compatible with the standard SQL query API. For more high-level description, and notes on future work, refer to #17139. This patch contains the following changes, grouped into packages. Controller (org.apache.druid.msq.dart.controller): The controller runs on Brokers. Main classes are, - DartSqlResource, which serves /druid/v2/sql/dart/. - DartSqlEngine and DartQueryMaker, the entry points from SQL that actually run the MSQ controller code. - DartControllerContext, which configures the MSQ controller. - DartMessageRelays, which sets up relays (see "message relays" below) to read messages from workers' DartControllerClients. - DartTableInputSpecSlicer, which assigns work based on a TimelineServerView. Worker (org.apache.druid.msq.dart.worker) The worker runs on Historicals. Main classes are, - DartWorkerResource, which supplies the regular MSQ WorkerResource, plus Dart-specific APIs. - DartWorkerRunner, which runs MSQ worker code. - DartWorkerContext, which configures the MSQ worker. - DartProcessingBuffersProvider, which provides processing buffers from sliced-up merge buffers. - DartDataSegmentProvider, which provides segments from the Historical's local cache. Message relays (org.apache.druid.messages): To avoid the need for Historicals to contact Brokers during a query, which would create opportunities for queries to get stuck, all connections are opened from Broker to Historical. This is made possible by a message relay system, where the relay server (worker) has an outbox of messages. The relay client (controller) connects to the outbox and retrieves messages. Code for this system lives in the "server" package to keep it separate from the MSQ extension and make it easier to maintain. The worker-to-controller ControllerClient is implemented using message relays. Other changes: - Controller: Added the method "hasWorker". Used by the ControllerMessageListener to notify the appropriate controllers when a worker fails. - WorkerResource: No longer tries to respond more than once in the "httpGetChannelData" API. This comes up when a response due to resolved future is ready at about the same time as a timeout occurs. - MSQTaskQueryMaker: Refactor to separate out some useful functions for reuse in DartQueryMaker. - SqlEngine: Add "queryContext" to "resultTypeForSelect" and "resultTypeForInsert". This allows the DartSqlEngine to modify result format based on whether a "fullReport" context parameter is set. - LimitedOutputStream: New utility class. Used when in "fullReport" mode. - TimelineServerView: Add getDruidServerMetadata as a performance optimization. - CliHistorical: Add SegmentWrangler, so it can query inline data, lookups, etc. - ServiceLocation: Add "fromUri" method, relocating some code from ServiceClientImpl. - FixedServiceLocator: New locator for a fixed set of service locations. Useful for URI locations. --- .../java/org/apache/druid/msq/dart/Dart.java | 37 ++ .../dart/DartResourcePermissionMapper.java | 46 ++ .../msq/dart/controller/ControllerHolder.java | 81 ++++ .../controller/ControllerMessageListener.java | 85 ++++ .../dart/controller/ControllerServerId.java | 116 +++++ .../controller/DartControllerContext.java | 244 ++++++++++ .../controller/DartControllerRegistry.java | 54 +++ .../DartControllerRegistryImpl.java | 68 +++ .../DartMessageRelayFactoryImpl.java | 62 +++ .../dart/controller/DartMessageRelays.java | 40 ++ .../controller/DartTableInputSpecSlicer.java | 282 +++++++++++ .../dart/controller/DartWorkerManager.java | 194 ++++++++ .../dart/controller/http/DartQueryInfo.java | 115 +++++ .../dart/controller/http/DartSqlResource.java | 233 ++++++++++ .../controller/http/GetQueriesResponse.java | 45 ++ .../messages/ControllerMessage.java | 49 ++ .../controller/messages/DoneReadingInput.java | 101 ++++ .../messages/PartialKeyStatistics.java | 118 +++++ .../controller/messages/ResultsComplete.java | 118 +++++ .../dart/controller/messages/WorkerError.java | 96 ++++ .../controller/messages/WorkerWarning.java | 96 ++++ .../dart/controller/sql/DartQueryMaker.java | 438 ++++++++++++++++++ .../dart/controller/sql/DartSqlClient.java | 42 ++ .../controller/sql/DartSqlClientFactory.java | 30 ++ .../sql/DartSqlClientFactoryImpl.java | 64 +++ .../controller/sql/DartSqlClientImpl.java | 57 +++ .../dart/controller/sql/DartSqlClients.java | 118 +++++ .../dart/controller/sql/DartSqlEngine.java | 170 +++++++ .../msq/dart/guice/DartControllerConfig.java | 44 ++ .../DartControllerMemoryManagementModule.java | 64 +++ .../msq/dart/guice/DartControllerModule.java | 159 +++++++ .../druid/msq/dart/guice/DartModules.java | 37 ++ .../msq/dart/guice/DartWorkerConfig.java | 53 +++ .../DartWorkerMemoryManagementModule.java | 100 ++++ .../msq/dart/guice/DartWorkerModule.java | 153 ++++++ .../msq/dart/worker/DartControllerClient.java | 137 ++++++ .../dart/worker/DartDataSegmentProvider.java | 111 +++++ .../msq/dart/worker/DartFrameContext.java | 178 +++++++ .../worker/DartProcessingBuffersProvider.java | 94 ++++ .../msq/dart/worker/DartWorkerClient.java | 152 ++++++ .../msq/dart/worker/DartWorkerContext.java | 246 ++++++++++ .../msq/dart/worker/DartWorkerFactory.java | 33 ++ .../dart/worker/DartWorkerFactoryImpl.java | 143 ++++++ .../dart/worker/DartWorkerRetryPolicy.java | 94 ++++ .../msq/dart/worker/DartWorkerRunner.java | 333 +++++++++++++ .../msq/dart/worker/QueryableDataSegment.java | 86 ++++ .../druid/msq/dart/worker/WorkerId.java | 157 +++++++ .../msq/dart/worker/http/DartWorkerInfo.java | 87 ++++ .../dart/worker/http/DartWorkerResource.java | 184 ++++++++ .../dart/worker/http/GetWorkersResponse.java | 44 ++ .../org/apache/druid/msq/exec/Controller.java | 5 + .../apache/druid/msq/exec/ControllerImpl.java | 10 + .../apache/druid/msq/exec/WorkerManager.java | 7 +- .../apache/druid/msq/rpc/WorkerResource.java | 15 +- .../druid/msq/sql/MSQTaskQueryMaker.java | 145 +++--- .../druid/msq/sql/MSQTaskSqlEngine.java | 12 +- .../msq/util/MSQTaskQueryMakerUtils.java | 5 +- ...rg.apache.druid.initialization.DruidModule | 4 + .../apache/druid/io/LimitedOutputStream.java | 74 +++ .../apache/druid/client/BrokerServerView.java | 14 + .../druid/client/TimelineServerView.java | 16 +- .../apache/druid/messages/MessageBatch.java | 112 +++++ .../messages/client/MessageListener.java | 50 ++ .../druid/messages/client/MessageRelay.java | 243 ++++++++++ .../messages/client/MessageRelayClient.java | 43 ++ .../client/MessageRelayClientImpl.java | 85 ++++ .../messages/client/MessageRelayFactory.java | 30 ++ .../client/MessageRelayFactoryImpl.java | 76 +++ .../druid/messages/client/MessageRelays.java | 146 ++++++ .../druid/messages/server/ClientMonitor.java | 82 ++++ .../messages/server/MessageRelayResource.java | 196 ++++++++ .../apache/druid/messages/server/Outbox.java | 66 +++ .../druid/messages/server/OutboxImpl.java | 218 +++++++++ .../apache/druid/rpc/FixedServiceLocator.java | 59 +++ .../apache/druid/rpc/ServiceClientImpl.java | 46 +- .../org/apache/druid/rpc/ServiceLocation.java | 64 +++ .../druid/rpc/ServiceClientImplTest.java | 8 - .../apache/druid/rpc/ServiceLocationTest.java | 8 + .../org/apache/druid/cli/CliHistorical.java | 2 + .../sql/calcite/planner/IngestHandler.java | 3 +- .../sql/calcite/planner/QueryHandler.java | 3 +- .../sql/calcite/run/NativeSqlEngine.java | 12 +- .../druid/sql/calcite/run/SqlEngine.java | 14 +- .../druid/sql/calcite/view/ViewSqlEngine.java | 12 +- .../apache/druid/sql/http/SqlResource.java | 35 +- .../sql/calcite/CalciteScanSignatureTest.java | 12 +- .../sql/calcite/IngestionTestSqlEngine.java | 12 +- .../calcite/util/TestTimelineServerView.java | 2 - 88 files changed, 7692 insertions(+), 142 deletions(-) create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/Dart.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/DartResourcePermissionMapper.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/ControllerHolder.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/ControllerMessageListener.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/ControllerServerId.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/DartControllerContext.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/DartControllerRegistry.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/DartControllerRegistryImpl.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/DartMessageRelayFactoryImpl.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/DartMessageRelays.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/DartTableInputSpecSlicer.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/DartWorkerManager.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/http/DartQueryInfo.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/http/DartSqlResource.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/http/GetQueriesResponse.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/messages/ControllerMessage.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/messages/DoneReadingInput.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/messages/PartialKeyStatistics.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/messages/ResultsComplete.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/messages/WorkerError.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/messages/WorkerWarning.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/sql/DartQueryMaker.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/sql/DartSqlClient.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/sql/DartSqlClientFactory.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/sql/DartSqlClientFactoryImpl.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/sql/DartSqlClientImpl.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/sql/DartSqlClients.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/sql/DartSqlEngine.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/guice/DartControllerConfig.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/guice/DartControllerMemoryManagementModule.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/guice/DartControllerModule.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/guice/DartModules.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/guice/DartWorkerConfig.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/guice/DartWorkerMemoryManagementModule.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/guice/DartWorkerModule.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartControllerClient.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartDataSegmentProvider.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartFrameContext.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartProcessingBuffersProvider.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartWorkerClient.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartWorkerContext.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartWorkerFactory.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartWorkerFactoryImpl.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartWorkerRetryPolicy.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartWorkerRunner.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/QueryableDataSegment.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/WorkerId.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/http/DartWorkerInfo.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/http/DartWorkerResource.java create mode 100644 extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/http/GetWorkersResponse.java create mode 100644 processing/src/main/java/org/apache/druid/io/LimitedOutputStream.java create mode 100644 server/src/main/java/org/apache/druid/messages/MessageBatch.java create mode 100644 server/src/main/java/org/apache/druid/messages/client/MessageListener.java create mode 100644 server/src/main/java/org/apache/druid/messages/client/MessageRelay.java create mode 100644 server/src/main/java/org/apache/druid/messages/client/MessageRelayClient.java create mode 100644 server/src/main/java/org/apache/druid/messages/client/MessageRelayClientImpl.java create mode 100644 server/src/main/java/org/apache/druid/messages/client/MessageRelayFactory.java create mode 100644 server/src/main/java/org/apache/druid/messages/client/MessageRelayFactoryImpl.java create mode 100644 server/src/main/java/org/apache/druid/messages/client/MessageRelays.java create mode 100644 server/src/main/java/org/apache/druid/messages/server/ClientMonitor.java create mode 100644 server/src/main/java/org/apache/druid/messages/server/MessageRelayResource.java create mode 100644 server/src/main/java/org/apache/druid/messages/server/Outbox.java create mode 100644 server/src/main/java/org/apache/druid/messages/server/OutboxImpl.java create mode 100644 server/src/main/java/org/apache/druid/rpc/FixedServiceLocator.java diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/Dart.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/Dart.java new file mode 100644 index 000000000000..33e239161ffe --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/Dart.java @@ -0,0 +1,37 @@ +/* + * 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.dart; + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * Binding annotation for implements of interfaces that are Dart (MSQ-on-Broker-and-Historicals) focused. + */ +@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD}) +@Retention(RetentionPolicy.RUNTIME) +@BindingAnnotation +public @interface Dart +{ +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/DartResourcePermissionMapper.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/DartResourcePermissionMapper.java new file mode 100644 index 000000000000..adedd811cbef --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/DartResourcePermissionMapper.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.dart; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.msq.rpc.ResourcePermissionMapper; +import org.apache.druid.server.security.Action; +import org.apache.druid.server.security.Resource; +import org.apache.druid.server.security.ResourceAction; + +import java.util.List; + +public class DartResourcePermissionMapper implements ResourcePermissionMapper +{ + @Override + public List getAdminPermissions() + { + return ImmutableList.of( + new ResourceAction(Resource.STATE_RESOURCE, Action.READ), + new ResourceAction(Resource.STATE_RESOURCE, Action.WRITE) + ); + } + + @Override + public List getQueryPermissions(String queryId) + { + return getAdminPermissions(); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/ControllerHolder.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/ControllerHolder.java new file mode 100644 index 000000000000..e3d4b60c096d --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/ControllerHolder.java @@ -0,0 +1,81 @@ +/* + * 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.dart.controller; + +import com.google.common.base.Preconditions; +import org.apache.druid.msq.exec.Controller; +import org.apache.druid.msq.util.MSQTaskQueryMakerUtils; +import org.joda.time.DateTime; + +import javax.annotation.Nullable; + +/** + * Holder for {@link Controller}, stored in {@link DartControllerRegistry}. + */ +public class ControllerHolder +{ + private final Controller controller; + private final String sqlQueryId; + private final String sql; + @Nullable + private final String identity; + private final DateTime startTime; + + public ControllerHolder( + final Controller controller, + final String sqlQueryId, + final String sql, + final String identity, + final DateTime startTime + ) + { + this.controller = Preconditions.checkNotNull(controller, "controller"); + this.sqlQueryId = Preconditions.checkNotNull(sqlQueryId, "sqlQueryId"); + this.sql = MSQTaskQueryMakerUtils.maskSensitiveJsonKeys(Preconditions.checkNotNull(sql, "sql")); + this.identity = identity; + this.startTime = Preconditions.checkNotNull(startTime, "startTime"); + } + + public Controller getController() + { + return controller; + } + + public String getSqlQueryId() + { + return sqlQueryId; + } + + public String getSql() + { + return sql; + } + + @Nullable + public String getIdentity() + { + return identity; + } + + public DateTime getStartTime() + { + return startTime; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/ControllerMessageListener.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/ControllerMessageListener.java new file mode 100644 index 000000000000..1e627562a3dc --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/ControllerMessageListener.java @@ -0,0 +1,85 @@ +/* + * 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.dart.controller; + +import com.google.inject.Inject; +import org.apache.druid.messages.client.MessageListener; +import org.apache.druid.msq.dart.controller.messages.ControllerMessage; +import org.apache.druid.msq.dart.worker.WorkerId; +import org.apache.druid.msq.exec.Controller; +import org.apache.druid.msq.indexing.error.MSQErrorReport; +import org.apache.druid.msq.indexing.error.WorkerFailedFault; +import org.apache.druid.server.DruidNode; + +/** + * Listener for worker-to-controller messages. Also responsible for calling + * {@link Controller#workerError(MSQErrorReport)} when a worker server goes away. + */ +public class ControllerMessageListener implements MessageListener +{ + private final DartControllerRegistry controllerRegistry; + + @Inject + public ControllerMessageListener(final DartControllerRegistry controllerRegistry) + { + this.controllerRegistry = controllerRegistry; + } + + @Override + public void serverAdded(DruidNode node) + { + // Fail workers when they're added, because when they're added, they shouldn't be running anything. If they are + // running something, cancel it. + workerFailed(node); + } + + @Override + public void serverRemoved(DruidNode node) + { + workerFailed(node); + } + + @Override + public void messageReceived(ControllerMessage message) + { + final Controller controller = controllerRegistry.get(message.getQueryId()); + if (controller != null) { + message.handle(controller); + } + } + + private void workerFailed(final DruidNode node) + { + for (final ControllerHolder holder : controllerRegistry.getAllHolders()) { + final Controller controller = holder.getController(); + final String workerId = WorkerId.fromDruidNode(node, controller.queryId()).toString(); + if (controller.hasWorker(workerId)) { + controller.workerError( + MSQErrorReport.fromFault( + workerId, + node.getHost(), + null, + new WorkerFailedFault(workerId, "Worker went offline") + ) + ); + } + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/ControllerServerId.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/ControllerServerId.java new file mode 100644 index 000000000000..4937a47f5eaf --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/ControllerServerId.java @@ -0,0 +1,116 @@ +/* + * 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.dart.controller; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonValue; +import com.google.common.base.Preconditions; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.msq.exec.Controller; +import org.apache.druid.server.DruidNode; + +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * Unique identifier for a particular run of the controller server. This does not correspond to any specific + * {@link Controller}; all controllers on the same server use the same ID. + */ +public class ControllerServerId +{ + private static final Pattern PATTERN = Pattern.compile("^(.+):(\\d+)$"); + + private final String host; + private final long epoch; + private final String fullString; + + public ControllerServerId(final String host, final long epoch) + { + this.host = Preconditions.checkNotNull(host, "host"); + this.epoch = epoch; + this.fullString = host + ':' + epoch; + } + + @JsonCreator + public static ControllerServerId fromString(final String s) + { + final Matcher matcher = PATTERN.matcher(s); + if (matcher.matches()) { + return new ControllerServerId(matcher.group(1), Long.parseLong(matcher.group(2))); + } else { + throw new IAE("Invalid controllerId[%s]", s); + } + } + + /** + * Host and port, from {@link DruidNode#getHostAndPortToUse()}, of the controller server. + */ + @JsonProperty + public String getHost() + { + return host; + } + + /** + * Epoch of the controller server. Increased every time the server reboots. This can be used to determine if + * the controller server being communicated with is the "same" controller server as one we know about with the + * same hostname. + */ + @JsonProperty + public long getEpoch() + { + return epoch; + } + + /** + * Returns whether this controllerId replaces another one, i.e., if the host is the same and epoch is greater. + */ + public boolean replaces(final ControllerServerId otherId) + { + return otherId.getHost().equals(host) && epoch > otherId.getEpoch(); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ControllerServerId that = (ControllerServerId) o; + return fullString.equals(that.fullString); + } + + @Override + public int hashCode() + { + return fullString.hashCode(); + } + + @Override + @JsonValue + public String toString() + { + return fullString; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/DartControllerContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/DartControllerContext.java new file mode 100644 index 000000000000..cdda56fac49b --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/DartControllerContext.java @@ -0,0 +1,244 @@ +/* + * 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.dart.controller; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Inject; +import com.google.inject.Injector; +import org.apache.druid.client.BrokerServerView; +import org.apache.druid.guice.annotations.EscalatedGlobal; +import org.apache.druid.guice.annotations.Json; +import org.apache.druid.guice.annotations.Self; +import org.apache.druid.guice.annotations.Smile; +import org.apache.druid.indexing.common.actions.TaskActionClient; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; +import org.apache.druid.msq.dart.worker.DartWorkerClient; +import org.apache.druid.msq.dart.worker.WorkerId; +import org.apache.druid.msq.exec.Controller; +import org.apache.druid.msq.exec.ControllerContext; +import org.apache.druid.msq.exec.ControllerMemoryParameters; +import org.apache.druid.msq.exec.MemoryIntrospector; +import org.apache.druid.msq.exec.WorkerFailureListener; +import org.apache.druid.msq.exec.WorkerManager; +import org.apache.druid.msq.indexing.IndexerControllerContext; +import org.apache.druid.msq.indexing.MSQSpec; +import org.apache.druid.msq.indexing.destination.TaskReportMSQDestination; +import org.apache.druid.msq.input.InputSpecSlicer; +import org.apache.druid.msq.kernel.controller.ControllerQueryKernelConfig; +import org.apache.druid.msq.querykit.QueryKit; +import org.apache.druid.msq.querykit.QueryKitSpec; +import org.apache.druid.msq.util.MultiStageQueryContext; +import org.apache.druid.query.Query; +import org.apache.druid.query.QueryContext; +import org.apache.druid.rpc.ServiceClientFactory; +import org.apache.druid.server.DruidNode; +import org.apache.druid.server.coordination.DruidServerMetadata; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +public class DartControllerContext implements ControllerContext +{ + /** + * Default for {@link ControllerQueryKernelConfig#getMaxConcurrentStages()}. + */ + public static final int DEFAULT_MAX_CONCURRENT_STAGES = 2; + + /** + * Default for {@link MultiStageQueryContext#getTargetPartitionsPerWorkerWithDefault(QueryContext, int)}. + */ + public static final int DEFAULT_TARGET_PARTITIONS_PER_WORKER = 1; + + /** + * Context parameter for maximum number of nonleaf workers. + */ + public static final String CTX_MAX_NON_LEAF_WORKER_COUNT = "maxNonLeafWorkers"; + + /** + * Default to scatter/gather style: fan in to a single worker after the leaf stage(s). + */ + public static final int DEFAULT_MAX_NON_LEAF_WORKER_COUNT = 1; + + private final Injector injector; + private final ObjectMapper jsonMapper; + private final ObjectMapper smileMapper; + private final DruidNode selfNode; + private final ControllerServerId selfId; + private final ServiceClientFactory serviceClientFactory; + private final BrokerServerView serverView; + private final MemoryIntrospector memoryIntrospector; + private final ServiceMetricEvent.Builder metricBuilder; + private final ServiceEmitter emitter; + + @Inject + public DartControllerContext( + final Injector injector, + @Json final ObjectMapper jsonMapper, + @Smile final ObjectMapper smileMapper, + @Self final DruidNode selfNode, + final ControllerServerId selfId, + @EscalatedGlobal final ServiceClientFactory serviceClientFactory, + final BrokerServerView serverView, + final ServiceEmitter emitter + ) + { + this.injector = injector; + this.jsonMapper = jsonMapper; + this.smileMapper = smileMapper; + this.selfNode = selfNode; + this.selfId = selfId; + this.serviceClientFactory = serviceClientFactory; + this.serverView = serverView; + this.memoryIntrospector = injector.getInstance(MemoryIntrospector.class); + this.metricBuilder = new ServiceMetricEvent.Builder(); + this.emitter = emitter; + } + + @Override + public ControllerQueryKernelConfig queryKernelConfig( + final String queryId, + final MSQSpec querySpec + ) + { + final List servers = serverView.getDruidServerMetadata(); + + // Lock in the list of workers when creating the kernel config. There is a race here: the serverView itself is + // allowed to float. If a segment moves to a new server that isn't part of our list after the WorkerManager is + // created, we won't be able to find a valid server for certain segments. This isn't expected to be a problem, + // since the serverView is referenced shortly after the worker list is created. + final List workerIds = new ArrayList<>(servers.size()); + for (final DruidServerMetadata server : servers) { + workerIds.add(WorkerId.fromDruidServerMetadata(server, queryId).toString()); + } + + // Shuffle workerIds, so we don't bias a single server to always be worker 0 (which tends to do more work). + Collections.shuffle(workerIds); + + final ControllerMemoryParameters memoryParameters = + ControllerMemoryParameters.createProductionInstance( + memoryIntrospector, + workerIds.size() + ); + + final int maxConcurrentStages = MultiStageQueryContext.getMaxConcurrentStagesWithDefault( + querySpec.getQuery().context(), + DEFAULT_MAX_CONCURRENT_STAGES + ); + + return ControllerQueryKernelConfig + .builder() + .controllerHost(selfNode.getHostAndPortToUse()) + .workerIds(workerIds) + .pipeline(maxConcurrentStages > 1) + .destination(TaskReportMSQDestination.instance()) + .maxConcurrentStages(maxConcurrentStages) + .maxRetainedPartitionSketchBytes(memoryParameters.getPartitionStatisticsMaxRetainedBytes()) + .workerContextMap(IndexerControllerContext.makeWorkerContextMap(querySpec, false, maxConcurrentStages)) + .build(); + } + + @Override + public ObjectMapper jsonMapper() + { + return jsonMapper; + } + + @Override + public Injector injector() + { + return injector; + } + + @Override + public void emitMetric(final String metric, final Number value) + { + emitter.emit(metricBuilder.setMetric(metric, value)); + } + + @Override + public DruidNode selfNode() + { + return selfNode; + } + + @Override + public InputSpecSlicer newTableInputSpecSlicer(WorkerManager workerManager) + { + return DartTableInputSpecSlicer.createFromWorkerManager((DartWorkerManager) workerManager, serverView); + } + + @Override + public TaskActionClient taskActionClient() + { + throw new UnsupportedOperationException(); + } + + @Override + public WorkerManager newWorkerManager( + String queryId, + MSQSpec querySpec, + ControllerQueryKernelConfig queryKernelConfig, + WorkerFailureListener workerFailureListener + ) + { + // We're ignoring WorkerFailureListener. Dart worker failures are routed into the controller by + // ControllerMessageListener, which receives a notification when a worker goes offline. + return new DartWorkerManager(queryKernelConfig.getWorkerIds(), newWorkerClient()); + } + + @Override + public DartWorkerClient newWorkerClient() + { + return new DartWorkerClient(serviceClientFactory, smileMapper, selfId); + } + + @Override + public void registerController(Controller controller, Closer closer) + { + // Nothing to do. + } + + @Override + public QueryKitSpec makeQueryKitSpec( + final QueryKit> queryKit, + final String queryId, + final MSQSpec querySpec, + final ControllerQueryKernelConfig queryKernelConfig + ) + { + final QueryContext queryContext = querySpec.getQuery().context(); + return new QueryKitSpec( + queryKit, + queryId, + queryKernelConfig.getWorkerIds().size(), + queryContext.getInt( + CTX_MAX_NON_LEAF_WORKER_COUNT, + DEFAULT_MAX_NON_LEAF_WORKER_COUNT + ), + MultiStageQueryContext.getTargetPartitionsPerWorkerWithDefault( + queryContext, + DEFAULT_TARGET_PARTITIONS_PER_WORKER + ) + ); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/DartControllerRegistry.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/DartControllerRegistry.java new file mode 100644 index 000000000000..6298e43cc353 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/DartControllerRegistry.java @@ -0,0 +1,54 @@ +/* + * 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.dart.controller; + +import org.apache.druid.error.DruidException; +import org.apache.druid.msq.exec.Controller; + +import javax.annotation.Nullable; +import java.util.Collection; + +/** + * Registry for actively-running {@link Controller}. + */ +public interface DartControllerRegistry +{ + /** + * Add a controller. Throws {@link DruidException} if a controller with the same {@link Controller#queryId()} is + * already registered. + */ + void register(ControllerHolder holder); + + /** + * Remove a controller from the registry. + */ + void remove(ControllerHolder holder); + + /** + * Return a specific controller, or null if it doesn't exist. + */ + @Nullable + Controller get(String queryId); + + /** + * Returns all actively-running {@link Controller}. + */ + Collection getAllHolders(); +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/DartControllerRegistryImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/DartControllerRegistryImpl.java new file mode 100644 index 000000000000..f5101222f1e2 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/DartControllerRegistryImpl.java @@ -0,0 +1,68 @@ +/* + * 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.dart.controller; + +import org.apache.druid.error.DruidException; +import org.apache.druid.msq.exec.Controller; + +import javax.annotation.Nullable; +import java.util.Collection; +import java.util.concurrent.ConcurrentHashMap; + +/** + * Production implementation of {@link DartControllerRegistry}. + */ +public class DartControllerRegistryImpl implements DartControllerRegistry +{ + private final ConcurrentHashMap controllerMap = new ConcurrentHashMap<>(); + + @Override + public void register(ControllerHolder holder) + { + if (controllerMap.putIfAbsent(holder.getController().queryId(), holder) != null) { + throw DruidException.defensive("Controller[%s] already registered", holder.getController().queryId()); + } + } + + @Override + public void remove(ControllerHolder holder) + { + // Remove only if the current mapping for the queryId is this specific controller. + controllerMap.remove(holder.getController().queryId(), holder); + } + + @Nullable + @Override + public Controller get(final String queryId) + { + final ControllerHolder holder = controllerMap.get(queryId); + if (holder != null) { + return holder.getController(); + } else { + return null; + } + } + + @Override + public Collection getAllHolders() + { + return controllerMap.values(); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/DartMessageRelayFactoryImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/DartMessageRelayFactoryImpl.java new file mode 100644 index 000000000000..9886010702a9 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/DartMessageRelayFactoryImpl.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.dart.controller; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Inject; +import org.apache.druid.guice.annotations.EscalatedGlobal; +import org.apache.druid.guice.annotations.Self; +import org.apache.druid.guice.annotations.Smile; +import org.apache.druid.messages.client.MessageRelay; +import org.apache.druid.messages.client.MessageRelayFactoryImpl; +import org.apache.druid.msq.dart.controller.messages.ControllerMessage; +import org.apache.druid.msq.dart.worker.http.DartWorkerResource; +import org.apache.druid.rpc.ServiceClientFactory; +import org.apache.druid.server.DruidNode; + +/** + * Specialized {@link MessageRelayFactoryImpl} for Dart controllers. + */ +public class DartMessageRelayFactoryImpl extends MessageRelayFactoryImpl +{ + @Inject + public DartMessageRelayFactoryImpl( + @Self DruidNode selfNode, + @EscalatedGlobal ServiceClientFactory clientFactory, + @Smile ObjectMapper smileMapper, + ControllerMessageListener messageListener + ) + { + super( + selfNode.getHostAndPortToUse(), + messageListener, + clientFactory, + DartWorkerResource.PATH + "/relay", + smileMapper, + ControllerMessage.class + ); + } + + @Override + public MessageRelay newRelay(DruidNode clientNode) + { + return super.newRelay(clientNode); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/DartMessageRelays.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/DartMessageRelays.java new file mode 100644 index 000000000000..b00c5acb42eb --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/DartMessageRelays.java @@ -0,0 +1,40 @@ +/* + * 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.dart.controller; + +import org.apache.druid.discovery.DruidNodeDiscoveryProvider; +import org.apache.druid.discovery.NodeRole; +import org.apache.druid.messages.client.MessageRelayFactory; +import org.apache.druid.messages.client.MessageRelays; +import org.apache.druid.msq.dart.controller.messages.ControllerMessage; + +/** + * Specialized {@link MessageRelays} for Dart controllers. + */ +public class DartMessageRelays extends MessageRelays +{ + public DartMessageRelays( + final DruidNodeDiscoveryProvider discoveryProvider, + final MessageRelayFactory messageRelayFactory + ) + { + super(discoveryProvider, messageRelayFactory, NodeRole.HISTORICAL); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/DartTableInputSpecSlicer.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/DartTableInputSpecSlicer.java new file mode 100644 index 000000000000..501ee5fc56b4 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/DartTableInputSpecSlicer.java @@ -0,0 +1,282 @@ +/* + * 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.dart.controller; + +import com.google.common.collect.FluentIterable; +import com.google.common.collect.ImmutableList; +import it.unimi.dsi.fastutil.objects.Object2IntMap; +import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap; +import org.apache.druid.client.TimelineServerView; +import org.apache.druid.client.selector.QueryableDruidServer; +import org.apache.druid.client.selector.ServerSelector; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.JodaUtils; +import org.apache.druid.msq.dart.worker.WorkerId; +import org.apache.druid.msq.exec.WorkerManager; +import org.apache.druid.msq.input.InputSlice; +import org.apache.druid.msq.input.InputSpec; +import org.apache.druid.msq.input.InputSpecSlicer; +import org.apache.druid.msq.input.NilInputSlice; +import org.apache.druid.msq.dart.worker.QueryableDataSegment; +import org.apache.druid.msq.input.table.RichSegmentDescriptor; +import org.apache.druid.msq.input.table.SegmentsInputSlice; +import org.apache.druid.msq.input.table.TableInputSpec; +import org.apache.druid.query.TableDataSource; +import org.apache.druid.query.filter.DimFilterUtils; +import org.apache.druid.server.coordination.DruidServerMetadata; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.TimelineLookup; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Set; +import java.util.function.ToIntFunction; + +/** + * Slices {@link TableInputSpec} into {@link SegmentsInputSlice} for persistent servers using + * {@link TimelineServerView}. + */ +public class DartTableInputSpecSlicer implements InputSpecSlicer +{ + private static final int UNKNOWN = -1; + + /** + * Worker host:port -> worker number. This is the reverse of the mapping from {@link WorkerManager#getWorkerIds()}. + */ + private final Object2IntMap reverseWorkers; + + /** + * Server view for identifying which segments exist and which servers (workers) have which segments. + */ + private final TimelineServerView serverView; + + DartTableInputSpecSlicer(final Object2IntMap reverseWorkers, final TimelineServerView serverView) + { + this.reverseWorkers = reverseWorkers; + this.serverView = serverView; + } + + public static DartTableInputSpecSlicer createFromWorkerManager( + final DartWorkerManager workerManager, + final TimelineServerView serverView + ) + { + final List workerIds = workerManager.getWorkerIds(); + + final Object2IntMap reverseWorkers = new Object2IntOpenHashMap<>(); + reverseWorkers.defaultReturnValue(UNKNOWN); + + for (int i = 0; i < workerIds.size(); i++) { + reverseWorkers.put(WorkerId.fromString(workerIds.get(i)).getHostAndPort(), i); + } + + return new DartTableInputSpecSlicer(reverseWorkers, serverView); + } + + @Override + public boolean canSliceDynamic(final InputSpec inputSpec) + { + return false; + } + + @Override + public List sliceStatic(final InputSpec inputSpec, final int maxNumSlices) + { + final TableInputSpec tableInputSpec = (TableInputSpec) inputSpec; + final TimelineLookup timeline = + serverView.getTimeline(new TableDataSource(tableInputSpec.getDataSource()).getAnalysis()).orElse(null); + + if (timeline == null) { + return Collections.emptyList(); + } + + final Set prunedSegments = + findQueryableDataSegments( + tableInputSpec, + timeline, + serverSelector -> findWorkerForServerSelector(serverSelector, maxNumSlices) + ); + + final List> assignments = new ArrayList<>(maxNumSlices); + while (assignments.size() < maxNumSlices) { + assignments.add(null); + } + + int nextRoundRobinWorker = 0; + for (final QueryableDataSegment segment : prunedSegments) { + final int worker; + if (segment.getWorkerNumber() == UNKNOWN) { + // Segment is not available on any worker. Assign to some worker, round-robin. Today, that server will throw + // an error about the segment not being findable, but perhaps one day, it will be able to load the segment + // on demand. + worker = nextRoundRobinWorker; + nextRoundRobinWorker = (nextRoundRobinWorker + 1) % maxNumSlices; + } else { + worker = segment.getWorkerNumber(); + } + + if (assignments.get(worker) == null) { + assignments.set(worker, new ArrayList<>()); + } + + assignments.get(worker).add(segment); + } + + return makeSegmentSlices(tableInputSpec.getDataSource(), assignments); + } + + @Override + public List sliceDynamic( + final InputSpec inputSpec, + final int maxNumSlices, + final int maxFilesPerSlice, + final long maxBytesPerSlice + ) + { + throw new UnsupportedOperationException(); + } + + private int findWorkerForServerSelector(final ServerSelector serverSelector, final int maxNumSlices) + { + final QueryableDruidServer server = serverSelector.pick(null); + + if (server == null) { + return UNKNOWN; + } + + final String serverHostAndPort = server.getServer().getHostAndPort(); + final int worker = reverseWorkers.getInt(serverHostAndPort); + + if (worker != UNKNOWN && worker < maxNumSlices) { + return worker; + } else { + return UNKNOWN; + } + } + + /** + * Pull the list of {@link DataSegment} that we should query, along with a clipping interval for each one, and + * a worker to get it from. + */ + public static Set findQueryableDataSegments( + final TableInputSpec tableInputSpec, + final TimelineLookup timeline, + final ToIntFunction toWorkersFunction + ) + { + final FluentIterable allSegments = + FluentIterable.from(JodaUtils.condenseIntervals(tableInputSpec.getIntervals())) + .transformAndConcat(timeline::lookup) + .transformAndConcat( + holder -> + FluentIterable + .from(holder.getObject()) + .filter(chunk -> shouldIncludeSegment(chunk.getObject())) + .transform(chunk -> { + final ServerSelector serverSelector = chunk.getObject(); + final DataSegment dataSegment = serverSelector.getSegment(); + final int worker = toWorkersFunction.applyAsInt(serverSelector); + return new QueryableDataSegment(dataSegment, holder.getInterval(), worker); + }) + .filter(segment -> !segment.getSegment().isTombstone()) + ); + + return DimFilterUtils.filterShards( + tableInputSpec.getFilter(), + tableInputSpec.getFilterFields(), + allSegments, + segment -> segment.getSegment().getShardSpec(), + new HashMap<>() + ); + } + + /** + * Create a list of {@link SegmentsInputSlice} and {@link NilInputSlice} assignments. + * + * @param dataSource datasource to read + * @param assignments list of assignment lists, one per slice + * + * @return a list of the same length as "assignments" + * + * @throws IllegalStateException if any provided segments do not match the provided datasource + */ + public static List makeSegmentSlices( + final String dataSource, + final List> assignments + ) + { + final List retVal = new ArrayList<>(assignments.size()); + + for (final List assignment : assignments) { + if (assignment == null || assignment.isEmpty()) { + retVal.add(NilInputSlice.INSTANCE); + } else { + final List descriptors = new ArrayList<>(); + for (final QueryableDataSegment segment : assignment) { + if (!dataSource.equals(segment.getSegment().getDataSource())) { + throw new ISE("Expected dataSource[%s] but got[%s]", dataSource, segment.getSegment().getDataSource()); + } + + descriptors.add(toRichSegmentDescriptor(segment)); + } + + retVal.add(new SegmentsInputSlice(dataSource, descriptors, ImmutableList.of())); + } + } + + return retVal; + } + + private static RichSegmentDescriptor toRichSegmentDescriptor(final QueryableDataSegment segment) + { + return new RichSegmentDescriptor( + segment.getSegment().getInterval(), + segment.getInterval(), + segment.getSegment().getVersion(), + segment.getSegment().getShardSpec().getPartitionNum() + ); + } + + /** + * Whether to include a segment from the timeline. Segments are included if they are not tombstones, and are also not + * purely realtime segments. + */ + private static boolean shouldIncludeSegment(final ServerSelector serverSelector) + { + if (serverSelector.getSegment().isTombstone()) { + return false; + } + + int numRealtimeServers = 0; + int numNonRealtimeServers = 0; + + for (final DruidServerMetadata server : serverSelector.getAllServers()) { + if (server.isSegmentReplicationTarget()) { + numNonRealtimeServers++; + } else { + numRealtimeServers++; + } + } + + return numNonRealtimeServers > 0 || (numNonRealtimeServers + numRealtimeServers == 0); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/DartWorkerManager.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/DartWorkerManager.java new file mode 100644 index 000000000000..114a86b6ba68 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/DartWorkerManager.java @@ -0,0 +1,194 @@ +/* + * 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.dart.controller; + +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.SettableFuture; +import it.unimi.dsi.fastutil.ints.Int2ObjectAVLTreeMap; +import it.unimi.dsi.fastutil.ints.Int2ObjectMap; +import it.unimi.dsi.fastutil.objects.Object2IntMap; +import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.error.DruidException; +import org.apache.druid.indexer.TaskState; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.msq.dart.worker.DartWorkerClient; +import org.apache.druid.msq.exec.ControllerContext; +import org.apache.druid.msq.exec.WorkerClient; +import org.apache.druid.msq.exec.WorkerManager; +import org.apache.druid.msq.exec.WorkerStats; +import org.apache.druid.msq.indexing.WorkerCount; +import org.apache.druid.utils.CloseableUtils; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; + +/** + * Dart implementation of the {@link WorkerManager} returned by {@link ControllerContext#newWorkerManager}. + * + * This manager does not actually launch workers. The workers are housed on long-lived servers outside of this + * manager's control. This manager merely reports on their existence. + */ +public class DartWorkerManager implements WorkerManager +{ + private static final Logger log = new Logger(DartWorkerManager.class); + + private final List workerIds; + private final DartWorkerClient workerClient; + private final Object2IntMap workerIdToNumber; + private final AtomicReference state = new AtomicReference<>(State.NEW); + private final SettableFuture stopFuture = SettableFuture.create(); + + enum State + { + NEW, + STARTED, + STOPPED + } + + public DartWorkerManager( + final List workerIds, + final DartWorkerClient workerClient + ) + { + this.workerIds = workerIds; + this.workerClient = workerClient; + this.workerIdToNumber = new Object2IntOpenHashMap<>(); + this.workerIdToNumber.defaultReturnValue(UNKNOWN_WORKER_NUMBER); + + for (int i = 0; i < workerIds.size(); i++) { + workerIdToNumber.put(workerIds.get(i), i); + } + } + + @Override + public ListenableFuture start() + { + if (!state.compareAndSet(State.NEW, State.STARTED)) { + throw new ISE("Cannot start from state[%s]", state.get()); + } + + return stopFuture; + } + + @Override + public void launchWorkersIfNeeded(int workerCount) + { + // Nothing to do, just validate the count. + if (workerCount > workerIds.size()) { + throw DruidException.defensive( + "Desired workerCount[%s] must be less than or equal to actual workerCount[%s]", + workerCount, + workerIds.size() + ); + } + } + + @Override + public void waitForWorkers(Set workerNumbers) + { + // Nothing to wait for, just validate the numbers. + for (final int workerNumber : workerNumbers) { + if (workerNumber >= workerIds.size()) { + throw DruidException.defensive( + "Desired workerNumber[%s] must be less than workerCount[%s]", + workerNumber, + workerIds.size() + ); + } + } + } + + @Override + public List getWorkerIds() + { + return workerIds; + } + + @Override + public WorkerCount getWorkerCount() + { + return new WorkerCount(workerIds.size(), 0); + } + + @Override + public int getWorkerNumber(String workerId) + { + return workerIdToNumber.getInt(workerId); + } + + @Override + public boolean isWorkerActive(String workerId) + { + return workerIdToNumber.containsKey(workerId); + } + + @Override + public Map> getWorkerStats() + { + final Int2ObjectMap> retVal = new Int2ObjectAVLTreeMap<>(); + + for (int i = 0; i < workerIds.size(); i++) { + retVal.put(i, Collections.singletonList(new WorkerStats(workerIds.get(0), TaskState.RUNNING, -1, -1))); + } + + return retVal; + } + + /** + * Stop method. Possibly signals workers to stop, but does not actually wait for them to exit. + * + * If "interrupt" is false, does nothing special (other than setting {@link #stopFuture}). The assumption is that + * a previous call to {@link WorkerClient#postFinish} would have caused the worker to exit. + * + * If "interrupt" is true, sends {@link DartWorkerClient#stopWorker(String)} to workers to stop the current query ID. + * + * @param interrupt whether to interrupt currently-running work + */ + @Override + public void stop(boolean interrupt) + { + if (state.compareAndSet(State.STARTED, State.STOPPED)) { + final List> futures = new ArrayList<>(); + + // Send stop commands to all workers. This ensures they exit promptly, and do not get left in a zombie state. + for (final String workerId : workerIds) { + futures.add(workerClient.stopWorker(workerId)); + } + + // Block until messages are acknowledged. + try { + FutureUtils.getUnchecked(Futures.successfulAsList(futures), false); + } + catch (Throwable ignored) { + // Suppress errors. + } + + CloseableUtils.closeAndSuppressExceptions(workerClient, e -> log.warn(e, "Failed to close workerClient")); + stopFuture.set(null); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/http/DartQueryInfo.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/http/DartQueryInfo.java new file mode 100644 index 000000000000..154ca97bf8fb --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/http/DartQueryInfo.java @@ -0,0 +1,115 @@ +/* + * 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.dart.controller.http; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.druid.msq.dart.controller.ControllerHolder; +import org.apache.druid.query.QueryContexts; +import org.joda.time.DateTime; + +/** + * Class included in {@link GetQueriesResponse}. + */ +public class DartQueryInfo +{ + private final String sqlQueryId; + private final String dartQueryId; + private final String sql; + private final String identity; + private final DateTime startTime; + + @JsonCreator + public DartQueryInfo( + @JsonProperty("sqlQueryId") final String sqlQueryId, + @JsonProperty("dartQueryId") final String dartQueryId, + @JsonProperty("sql") final String sql, + @JsonProperty("identity") final String identity, + @JsonProperty("startTime") final DateTime startTime + ) + { + this.sqlQueryId = Preconditions.checkNotNull(sqlQueryId, "sqlQueryId"); + this.dartQueryId = Preconditions.checkNotNull(dartQueryId, "dartQueryId"); + this.sql = sql; + this.identity = identity; + this.startTime = startTime; + } + + public static DartQueryInfo fromControllerHolder(final ControllerHolder holder) + { + return new DartQueryInfo( + holder.getSqlQueryId(), + holder.getController().queryId(), + holder.getSql(), + holder.getIdentity(), + holder.getStartTime() + ); + } + + /** + * The {@link QueryContexts#CTX_SQL_QUERY_ID} provided by the user, or generated by the system. + */ + @JsonProperty + public String getSqlQueryId() + { + return sqlQueryId; + } + + /** + * Dart query ID generated by the system. Globally unique. + */ + @JsonProperty + public String getDartQueryId() + { + return dartQueryId; + } + + /** + * SQL string for this query. + */ + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public String getSql() + { + return sql; + } + + /** + * User that issued this query. + */ + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public String getIdentity() + { + return identity; + } + + /** + * Time this query was started. + */ + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public DateTime getStartTime() + { + return startTime; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/http/DartSqlResource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/http/DartSqlResource.java new file mode 100644 index 000000000000..59070bdb0995 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/http/DartSqlResource.java @@ -0,0 +1,233 @@ +/* + * 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.dart.controller.http; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import com.google.common.util.concurrent.Futures; +import com.google.inject.Inject; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.guice.annotations.Self; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.msq.dart.Dart; +import org.apache.druid.msq.dart.controller.DartControllerRegistry; +import org.apache.druid.msq.dart.controller.sql.DartSqlClients; +import org.apache.druid.msq.dart.controller.sql.DartSqlEngine; +import org.apache.druid.msq.exec.Controller; +import org.apache.druid.msq.rpc.MSQResourceUtils; +import org.apache.druid.msq.rpc.ResourcePermissionMapper; +import org.apache.druid.query.DefaultQueryConfig; +import org.apache.druid.server.DruidNode; +import org.apache.druid.server.ResponseContextConfig; +import org.apache.druid.server.initialization.ServerConfig; +import org.apache.druid.server.security.Access; +import org.apache.druid.server.security.AuthorizationUtils; +import org.apache.druid.server.security.AuthorizerMapper; +import org.apache.druid.sql.HttpStatement; +import org.apache.druid.sql.SqlLifecycleManager; +import org.apache.druid.sql.SqlStatementFactory; +import org.apache.druid.sql.http.SqlQuery; +import org.apache.druid.sql.http.SqlResource; + +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 java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.stream.Collectors; + +@Path(DartSqlResource.PATH + '/') +public class DartSqlResource extends SqlResource +{ + public static final String PATH = "/druid/v2/sql/dart"; + + private static final Logger log = new Logger(DartSqlResource.class); + + private final DartControllerRegistry controllerRegistry; + private final SqlLifecycleManager sqlLifecycleManager; + private final DartSqlClients sqlClients; + private final AuthorizerMapper authorizerMapper; + private final ResourcePermissionMapper permissionMapper; + private final DefaultQueryConfig dartQueryConfig; + + @Inject + public DartSqlResource( + final ObjectMapper jsonMapper, + final AuthorizerMapper authorizerMapper, + @Dart final SqlStatementFactory sqlStatementFactory, + final DartControllerRegistry controllerRegistry, + final SqlLifecycleManager sqlLifecycleManager, + final DartSqlClients sqlClients, + final ServerConfig serverConfig, + final ResponseContextConfig responseContextConfig, + @Dart final ResourcePermissionMapper permissionMapper, + @Self final DruidNode selfNode, + @Dart final DefaultQueryConfig dartQueryConfig + ) + { + super( + jsonMapper, + authorizerMapper, + sqlStatementFactory, + sqlLifecycleManager, + serverConfig, + responseContextConfig, + selfNode + ); + this.controllerRegistry = controllerRegistry; + this.sqlLifecycleManager = sqlLifecycleManager; + this.sqlClients = sqlClients; + this.authorizerMapper = authorizerMapper; + this.permissionMapper = permissionMapper; + this.dartQueryConfig = dartQueryConfig; + } + + /** + * API that allows callers to check if this resource is installed without actually issuing a query. If installed, + * this call returns 200 OK. If not installed, callers get 404 Not Found. + */ + @GET + @Path("/enabled") + @Produces(MediaType.APPLICATION_JSON) + public Response doGetEnabled(@Context final HttpServletRequest request) + { + AuthorizationUtils.setRequestAuthorizationAttributeIfNeeded(request); + return Response.ok(ImmutableMap.of("enabled", true)).build(); + } + + /** + * API to list all running queries. + * + * @param selfOnly if true, return queries running on this server. If false, return queries running on all servers. + * @param req http request + */ + @GET + @Produces(MediaType.APPLICATION_JSON) + public GetQueriesResponse doGetRunningQueries( + @QueryParam("selfOnly") final String selfOnly, + @Context final HttpServletRequest req + ) + { + MSQResourceUtils.authorizeAdminRequest(permissionMapper, authorizerMapper, req); + + final List queries = + controllerRegistry.getAllHolders() + .stream() + .map(DartQueryInfo::fromControllerHolder) + .collect(Collectors.toList()); + + // Add queries from all other servers, if "selfOnly" is not set. + if (selfOnly == null) { + final List otherQueries = FutureUtils.getUnchecked( + Futures.allAsList(Iterables.transform(sqlClients.getAllClients(), client -> client.getRunningQueries(true))), + true + ); + + for (final GetQueriesResponse response : otherQueries) { + queries.addAll(response.getQueries()); + } + } + + return new GetQueriesResponse(queries); + } + + /** + * API to issue a query. + */ + @POST + @Produces(MediaType.APPLICATION_JSON) + @Consumes(MediaType.APPLICATION_JSON) + @Override + public Response doPost( + final SqlQuery sqlQuery, + @Context final HttpServletRequest req + ) + { + final Map context = new HashMap<>(sqlQuery.getContext()); + + // Default context keys from dartQueryConfig. + for (Map.Entry entry : dartQueryConfig.getContext().entrySet()) { + context.putIfAbsent(entry.getKey(), entry.getValue()); + } + + // Dart queryId must be globally unique; cannot use user-provided sqlQueryId or queryId. + final String dartQueryId = UUID.randomUUID().toString(); + context.put(DartSqlEngine.CTX_DART_QUERY_ID, dartQueryId); + + return super.doPost(sqlQuery.withOverridenContext(context), req); + } + + /** + * API to cancel a query. + */ + @DELETE + @Path("{id}") + @Produces(MediaType.APPLICATION_JSON) + @Override + public Response cancelQuery( + @PathParam("id") String sqlQueryId, + @Context final HttpServletRequest req + ) + { + log.debug("Received cancel request for query[%s]", sqlQueryId); + + List cancelables = sqlLifecycleManager.getAll(sqlQueryId); + if (cancelables.isEmpty()) { + return Response.status(Response.Status.NOT_FOUND).build(); + } + + final Access access = authorizeCancellation(req, cancelables); + + if (access.isAllowed()) { + sqlLifecycleManager.removeAll(sqlQueryId, cancelables); + + // Don't call cancel() on the cancelables. That just cancels native queries, which is useless here. Instead, + // get the controller and stop it. + boolean found = false; + for (SqlLifecycleManager.Cancelable cancelable : cancelables) { + final HttpStatement stmt = (HttpStatement) cancelable; + final Object dartQueryId = stmt.context().get(DartSqlEngine.CTX_DART_QUERY_ID); + if (dartQueryId instanceof String) { + final Controller controller = controllerRegistry.get((String) dartQueryId); + if (controller != null) { + found = true; + controller.stop(); + } + } + } + + return Response.status(found ? Response.Status.ACCEPTED : Response.Status.NOT_FOUND).build(); + } else { + return Response.status(Response.Status.FORBIDDEN).build(); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/http/GetQueriesResponse.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/http/GetQueriesResponse.java new file mode 100644 index 000000000000..c98e492ccc62 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/http/GetQueriesResponse.java @@ -0,0 +1,45 @@ +/* + * 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.dart.controller.http; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.List; + +/** + * Class returned by {@link DartSqlResource#doGetRunningQueries}, the "list all queries" API. + */ +public class GetQueriesResponse +{ + private final List queries; + + @JsonCreator + public GetQueriesResponse(@JsonProperty("queries") List queries) + { + this.queries = queries; + } + + @JsonProperty + public List getQueries() + { + return queries; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/messages/ControllerMessage.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/messages/ControllerMessage.java new file mode 100644 index 000000000000..454e23bbc9c1 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/messages/ControllerMessage.java @@ -0,0 +1,49 @@ +/* + * 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.dart.controller.messages; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import org.apache.druid.msq.dart.worker.DartControllerClient; +import org.apache.druid.msq.exec.Controller; + +/** + * Messages sent from worker to controller by {@link DartControllerClient}. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") +@JsonSubTypes({ + @JsonSubTypes.Type(value = PartialKeyStatistics.class, name = "partialKeyStatistics"), + @JsonSubTypes.Type(value = DoneReadingInput.class, name = "doneReadingInput"), + @JsonSubTypes.Type(value = ResultsComplete.class, name = "resultsComplete"), + @JsonSubTypes.Type(value = WorkerError.class, name = "workerError"), + @JsonSubTypes.Type(value = WorkerWarning.class, name = "workerWarning") +}) +public interface ControllerMessage +{ + /** + * Query ID, to identify the controller that is being contacted. + */ + String getQueryId(); + + /** + * Handler for this message, which calls an appropriate method on {@link Controller}. + */ + void handle(Controller controller); +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/messages/DoneReadingInput.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/messages/DoneReadingInput.java new file mode 100644 index 000000000000..e74e5a0d1bb7 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/messages/DoneReadingInput.java @@ -0,0 +1,101 @@ +/* + * 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.dart.controller.messages; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.druid.msq.exec.Controller; +import org.apache.druid.msq.exec.ControllerClient; +import org.apache.druid.msq.kernel.StageId; + +import java.util.Objects; + +/** + * Message for {@link ControllerClient#postDoneReadingInput}. + */ +public class DoneReadingInput implements ControllerMessage +{ + private final StageId stageId; + private final int workerNumber; + + @JsonCreator + public DoneReadingInput( + @JsonProperty("stage") final StageId stageId, + @JsonProperty("worker") final int workerNumber + ) + { + this.stageId = Preconditions.checkNotNull(stageId, "stageId"); + this.workerNumber = workerNumber; + } + + @Override + public String getQueryId() + { + return stageId.getQueryId(); + } + + @JsonProperty("stage") + public StageId getStageId() + { + return stageId; + } + + @JsonProperty("worker") + public int getWorkerNumber() + { + return workerNumber; + } + + @Override + public void handle(Controller controller) + { + controller.doneReadingInput(stageId.getStageNumber(), workerNumber); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DoneReadingInput that = (DoneReadingInput) o; + return workerNumber == that.workerNumber + && Objects.equals(stageId, that.stageId); + } + + @Override + public int hashCode() + { + return Objects.hash(stageId, workerNumber); + } + + @Override + public String toString() + { + return "DoneReadingInput{" + + "stageId=" + stageId + + ", workerNumber=" + workerNumber + + '}'; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/messages/PartialKeyStatistics.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/messages/PartialKeyStatistics.java new file mode 100644 index 000000000000..1aa3bcb040e4 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/messages/PartialKeyStatistics.java @@ -0,0 +1,118 @@ +/* + * 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.dart.controller.messages; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.druid.msq.exec.Controller; +import org.apache.druid.msq.exec.ControllerClient; +import org.apache.druid.msq.kernel.StageId; +import org.apache.druid.msq.statistics.PartialKeyStatisticsInformation; + +import java.util.Objects; + +/** + * Message for {@link ControllerClient#postPartialKeyStatistics}. + */ +public class PartialKeyStatistics implements ControllerMessage +{ + private final StageId stageId; + private final int workerNumber; + private final PartialKeyStatisticsInformation payload; + + @JsonCreator + public PartialKeyStatistics( + @JsonProperty("stage") final StageId stageId, + @JsonProperty("worker") final int workerNumber, + @JsonProperty("payload") final PartialKeyStatisticsInformation payload + ) + { + this.stageId = Preconditions.checkNotNull(stageId, "stageId"); + this.workerNumber = workerNumber; + this.payload = payload; + } + + @Override + public String getQueryId() + { + return stageId.getQueryId(); + } + + @JsonProperty("stage") + public StageId getStageId() + { + return stageId; + } + + @JsonProperty("worker") + public int getWorkerNumber() + { + return workerNumber; + } + + @JsonProperty + public PartialKeyStatisticsInformation getPayload() + { + return payload; + } + + + @Override + public void handle(Controller controller) + { + controller.updatePartialKeyStatisticsInformation( + stageId.getStageNumber(), + workerNumber, + payload + ); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PartialKeyStatistics that = (PartialKeyStatistics) o; + return workerNumber == that.workerNumber + && Objects.equals(stageId, that.stageId) + && Objects.equals(payload, that.payload); + } + + @Override + public int hashCode() + { + return Objects.hash(stageId, workerNumber, payload); + } + + @Override + public String toString() + { + return "PartialKeyStatistics{" + + "stageId=" + stageId + + ", workerNumber=" + workerNumber + + ", payload=" + payload + + '}'; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/messages/ResultsComplete.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/messages/ResultsComplete.java new file mode 100644 index 000000000000..58822a357265 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/messages/ResultsComplete.java @@ -0,0 +1,118 @@ +/* + * 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.dart.controller.messages; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.druid.msq.exec.Controller; +import org.apache.druid.msq.exec.ControllerClient; +import org.apache.druid.msq.kernel.StageId; + +import javax.annotation.Nullable; +import java.util.Objects; + +/** + * Message for {@link ControllerClient#postResultsComplete}. + */ +public class ResultsComplete implements ControllerMessage +{ + private final StageId stageId; + private final int workerNumber; + + @Nullable + private final Object resultObject; + + @JsonCreator + public ResultsComplete( + @JsonProperty("stage") final StageId stageId, + @JsonProperty("worker") final int workerNumber, + @Nullable @JsonProperty("result") final Object resultObject + ) + { + this.stageId = Preconditions.checkNotNull(stageId, "stageId"); + this.workerNumber = workerNumber; + this.resultObject = resultObject; + } + + @Override + public String getQueryId() + { + return stageId.getQueryId(); + } + + @JsonProperty("stage") + public StageId getStageId() + { + return stageId; + } + + @JsonProperty("worker") + public int getWorkerNumber() + { + return workerNumber; + } + + @Nullable + @JsonProperty("result") + @JsonInclude(JsonInclude.Include.NON_NULL) + public Object getResultObject() + { + return resultObject; + } + + @Override + public void handle(Controller controller) + { + controller.resultsComplete(stageId.getQueryId(), stageId.getStageNumber(), workerNumber, resultObject); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ResultsComplete that = (ResultsComplete) o; + return workerNumber == that.workerNumber + && Objects.equals(stageId, that.stageId) + && Objects.equals(resultObject, that.resultObject); + } + + @Override + public int hashCode() + { + return Objects.hash(stageId, workerNumber, resultObject); + } + + @Override + public String toString() + { + return "ResultsComplete{" + + "stageId=" + stageId + + ", workerNumber=" + workerNumber + + ", resultObject=" + resultObject + + '}'; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/messages/WorkerError.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/messages/WorkerError.java new file mode 100644 index 000000000000..b89cfb356a36 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/messages/WorkerError.java @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.dart.controller.messages; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.druid.msq.exec.Controller; +import org.apache.druid.msq.exec.ControllerClient; +import org.apache.druid.msq.indexing.error.MSQErrorReport; + +import java.util.Objects; + +/** + * Message for {@link ControllerClient#postWorkerError}. + */ +public class WorkerError implements ControllerMessage +{ + private final String queryId; + private final MSQErrorReport errorWrapper; + + @JsonCreator + public WorkerError( + @JsonProperty("queryId") String queryId, + @JsonProperty("error") MSQErrorReport errorWrapper + ) + { + this.queryId = Preconditions.checkNotNull(queryId, "queryId"); + this.errorWrapper = Preconditions.checkNotNull(errorWrapper, "error"); + } + + @Override + @JsonProperty + public String getQueryId() + { + return queryId; + } + + @JsonProperty("error") + public MSQErrorReport getErrorWrapper() + { + return errorWrapper; + } + + @Override + public void handle(Controller controller) + { + controller.workerError(errorWrapper); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + WorkerError that = (WorkerError) o; + return Objects.equals(queryId, that.queryId) + && Objects.equals(errorWrapper, that.errorWrapper); + } + + @Override + public int hashCode() + { + return Objects.hash(queryId, errorWrapper); + } + + @Override + public String toString() + { + return "WorkerError{" + + "queryId='" + queryId + '\'' + + ", errorWrapper=" + errorWrapper + + '}'; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/messages/WorkerWarning.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/messages/WorkerWarning.java new file mode 100644 index 000000000000..aa2ff6643131 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/messages/WorkerWarning.java @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.dart.controller.messages; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.druid.msq.exec.Controller; +import org.apache.druid.msq.exec.ControllerClient; +import org.apache.druid.msq.indexing.error.MSQErrorReport; + +import java.util.List; +import java.util.Objects; + +/** + * Message for {@link ControllerClient#postWorkerWarning}. + */ +public class WorkerWarning implements ControllerMessage +{ + private final String queryId; + private final List errorWrappers; + + @JsonCreator + public WorkerWarning( + @JsonProperty("queryId") String queryId, + @JsonProperty("errors") List errorWrappers + ) + { + this.queryId = Preconditions.checkNotNull(queryId, "queryId"); + this.errorWrappers = Preconditions.checkNotNull(errorWrappers, "error"); + } + + @Override + @JsonProperty + public String getQueryId() + { + return queryId; + } + + @JsonProperty("errors") + public List getErrorWrappers() + { + return errorWrappers; + } + + @Override + public void handle(Controller controller) + { + controller.workerWarning(errorWrappers); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + WorkerWarning that = (WorkerWarning) o; + return Objects.equals(queryId, that.queryId) && Objects.equals(errorWrappers, that.errorWrappers); + } + + @Override + public int hashCode() + { + return Objects.hash(queryId, errorWrappers); + } + + @Override + public String toString() + { + return "WorkerWarning{" + + "queryId='" + queryId + '\'' + + ", errorWrappers=" + errorWrappers + + '}'; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/sql/DartQueryMaker.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/sql/DartQueryMaker.java new file mode 100644 index 000000000000..008b34582f03 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/sql/DartQueryMaker.java @@ -0,0 +1,438 @@ +/* + * 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.dart.controller.sql; + +import com.google.errorprone.annotations.concurrent.GuardedBy; +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.druid.io.LimitedOutputStream; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.Either; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.guava.BaseSequence; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.java.util.common.jackson.JacksonUtils; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.msq.dart.controller.ControllerHolder; +import org.apache.druid.msq.dart.controller.DartControllerRegistry; +import org.apache.druid.msq.dart.guice.DartControllerConfig; +import org.apache.druid.msq.exec.Controller; +import org.apache.druid.msq.exec.ControllerContext; +import org.apache.druid.msq.exec.ControllerImpl; +import org.apache.druid.msq.exec.QueryListener; +import org.apache.druid.msq.exec.ResultsContext; +import org.apache.druid.msq.indexing.MSQSpec; +import org.apache.druid.msq.indexing.TaskReportQueryListener; +import org.apache.druid.msq.indexing.destination.TaskReportMSQDestination; +import org.apache.druid.msq.indexing.error.MSQException; +import org.apache.druid.msq.indexing.report.MSQResultsReport; +import org.apache.druid.msq.indexing.report.MSQStatusReport; +import org.apache.druid.msq.indexing.report.MSQTaskReportPayload; +import org.apache.druid.msq.sql.MSQTaskQueryMaker; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.server.QueryResponse; +import org.apache.druid.sql.calcite.planner.PlannerContext; +import org.apache.druid.sql.calcite.rel.DruidQuery; +import org.apache.druid.sql.calcite.run.QueryMaker; +import org.apache.druid.sql.calcite.run.SqlResults; +import org.apache.druid.utils.CloseableUtils; + +import javax.annotation.Nullable; +import java.io.ByteArrayOutputStream; +import java.io.Closeable; +import java.io.IOException; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.NoSuchElementException; +import java.util.Optional; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.Semaphore; +import java.util.stream.Collectors; + +/** + * SQL {@link QueryMaker}. Executes queries in two ways, depending on whether the user asked for a full report. + * + * When including a full report, the controller runs in the SQL planning thread (typically an HTTP thread) using + * the method {@link #runWithReport(ControllerHolder)}. The entire response is buffered in memory, up to + * {@link DartControllerConfig#getMaxQueryReportSize()}. + * + * When not including a full report, the controller runs in {@link #controllerExecutor} and results are streamed + * back to the user through {@link ResultIterator}. There is no limit to the size of the returned results. + */ +public class DartQueryMaker implements QueryMaker +{ + private static final Logger log = new Logger(DartQueryMaker.class); + + private final List> fieldMapping; + private final ControllerContext controllerContext; + private final PlannerContext plannerContext; + + /** + * Controller registry, used to register and remove controllers as they start and finish. + */ + private final DartControllerRegistry controllerRegistry; + + /** + * Controller config. + */ + private final DartControllerConfig controllerConfig; + + /** + * Semaphore to prevent more than {@link DartControllerConfig#getConcurrentQueries()} controllers running at once. + */ + private final Semaphore controllerSemaphore; + + /** + * Executor for {@link #runWithoutReport(ControllerHolder)}. + */ + private final ExecutorService controllerExecutor; + + public DartQueryMaker( + List> fieldMapping, + ControllerContext controllerContext, + PlannerContext plannerContext, + DartControllerRegistry controllerRegistry, + DartControllerConfig controllerConfig, + Semaphore controllerSemaphore, + ExecutorService controllerExecutor + ) + { + this.fieldMapping = fieldMapping; + this.controllerContext = controllerContext; + this.plannerContext = plannerContext; + this.controllerRegistry = controllerRegistry; + this.controllerConfig = controllerConfig; + this.controllerSemaphore = controllerSemaphore; + this.controllerExecutor = controllerExecutor; + } + + @Override + public QueryResponse runQuery(DruidQuery druidQuery) + { + final MSQSpec querySpec = MSQTaskQueryMaker.makeQuerySpec( + null, + druidQuery, + fieldMapping, + plannerContext, + null // Only used for DML, which this isn't + ); + final List> types = + MSQTaskQueryMaker.getTypes(druidQuery, fieldMapping, plannerContext); + + final ControllerImpl controller = new ControllerImpl( + druidQuery.getQuery().context().getString(DartSqlEngine.CTX_DART_QUERY_ID), + querySpec, + new ResultsContext( + types.stream().map(p -> p.lhs).collect(Collectors.toList()), + SqlResults.Context.fromPlannerContext(plannerContext) + ), + controllerContext + ); + + final ControllerHolder controllerHolder = new ControllerHolder( + controller, + plannerContext.getSqlQueryId(), + plannerContext.getSql(), + plannerContext.getAuthenticationResult().getIdentity(), + DateTimes.nowUtc() + ); + + final boolean fullReport = druidQuery.getQuery().context().getBoolean( + DartSqlEngine.CTX_FULL_REPORT, + DartSqlEngine.CTX_FULL_REPORT_DEFAULT + ); + + // Register controller before acquiring the semaphore, so it shows up in "active controllers" lists. + controllerRegistry.register(controllerHolder); + + try { + controllerSemaphore.acquire(); + } + catch (InterruptedException e) { + // Error while acquiring semaphore; unregister controller. + controllerRegistry.remove(controllerHolder); + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + + try { + final Sequence results = + fullReport ? runWithReport(controllerHolder) : runWithoutReport(controllerHolder); + return QueryResponse.withEmptyContext(results); + } + catch (Throwable e) { + // Error while creating the result sequence; release semaphore and unregister controller. + controllerSemaphore.release(); + controllerRegistry.remove(controllerHolder); + throw e; + } + } + + /** + * Run a query and return the full report, buffered in memory up to + * {@link DartControllerConfig#getMaxQueryReportSize()}. + */ + private Sequence runWithReport(final ControllerHolder controllerHolder) + { + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + final TaskReportQueryListener queryListener = new TaskReportQueryListener( + TaskReportMSQDestination.instance(), + () -> new LimitedOutputStream( + baos, + controllerConfig.getMaxQueryReportSize(), + limit -> StringUtils.format( + "maxQueryReportSize[%,d] exceeded. " + + "Try limiting the result set for your query, or run it with %s[false]", + limit, + DartSqlEngine.CTX_FULL_REPORT + ) + ), + plannerContext.getJsonMapper(), + controllerHolder.getController().queryId(), + Collections.emptyMap() + ); + + try { + controllerHolder.getController().run(queryListener); + + final Map reportAsMap = + plannerContext.getJsonMapper().readValue(baos.toByteArray(), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT); + return Sequences.simple(Collections.singletonList(new Object[]{reportAsMap})); + } + catch (Exception e) { + throw new RuntimeException(e); + } + finally { + controllerSemaphore.release(); + controllerRegistry.remove(controllerHolder); + } + } + + /** + * Run a query and return the results only, streamed back using {@link ResultIteratorMaker}. + */ + private Sequence runWithoutReport(final ControllerHolder controllerHolder) + { + return new BaseSequence<>(new ResultIteratorMaker(controllerHolder)); + } + + /** + * Helper for {@link #runWithoutReport(ControllerHolder)}. + */ + class ResultIteratorMaker implements BaseSequence.IteratorMaker + { + private final ControllerHolder controllerHolder; + + public ResultIteratorMaker(ControllerHolder holder) + { + this.controllerHolder = holder; + } + + @Override + public ResultIterator make() + { + final ResultIterator iter = new ResultIterator(); + + // This separate thread is needed because we need to return a Sequence that streams results. If the + // QueryMaker interface was changed to use an object that pushes out results rather than returning a Sequence, + // then we wouldn't need this separate thread. + final Future controllerFuture = controllerExecutor.submit(() -> { + final Controller controller = controllerHolder.getController(); + final String threadName = Thread.currentThread().getName(); + + try { + Thread.currentThread().setName( + StringUtils.format( + "%s-sqlQueryId[%s]-queryId[%s]", + threadName, + plannerContext.getSqlQueryId(), + controller.queryId() + ) + ); + controller.run(iter); + } + catch (Exception e) { + log.warn( + e, + "Controller failed for sqlQueryId[%s], controllerHost[%s]", + plannerContext.getSqlQueryId(), + controller.queryId() + ); + } + finally { + controllerSemaphore.release(); + controllerRegistry.remove(controllerHolder); + Thread.currentThread().setName(threadName); + } + }); + + iter.attach(() -> { + controllerFuture.cancel(false); + + if (!iter.complete) { + controllerHolder.getController().stop(); + } + }); + + return iter; + } + + @Override + public void cleanup(final ResultIterator iterFromMake) + { + CloseableUtils.closeAndWrapExceptions(iterFromMake); + } + } + + /** + * Helper for {@link ResultIteratorMaker}, which is in turn a helper for {@link #runWithoutReport(ControllerHolder)}. + */ + static class ResultIterator implements Iterator, QueryListener, Closeable + { + /** + * Number of rows to buffer from {@link #onResultRow(Object[])}. + */ + private static final int BUFFER_SIZE = 128; + + /** + * Empty optional signifies results are complete. + */ + private final BlockingQueue> rowBuffer = new ArrayBlockingQueue<>(BUFFER_SIZE); + + /** + * Only accessed by {@link Iterator} methods, so no need to be thread-safe. + */ + @Nullable + private Either current; + + @GuardedBy("closer") + private final Closer closer = Closer.create(); + + private volatile boolean complete; + + @Override + public boolean hasNext() + { + return populateAndReturnCurrent().isPresent(); + } + + @Override + public Object[] next() + { + final Object[] retVal = populateAndReturnCurrent().orElseThrow(NoSuchElementException::new); + current = null; + return retVal; + } + + private Optional populateAndReturnCurrent() + { + if (current == null) { + try { + current = rowBuffer.take(); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + } + + return Optional.ofNullable(current.valueOrThrow()); + } + + @Override + public boolean readResults() + { + return !complete; + } + + @Override + public void onResultsStart( + final List signature, + @Nullable final List sqlTypeNames + ) + { + // Nothing to do. + } + + @Override + public boolean onResultRow(Object[] row) + { + try { + rowBuffer.put(Either.value(row)); + return !complete; + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + } + + @Override + public void onResultsComplete() + { + // Nothing to do. + } + + @Override + public void onQueryComplete(MSQTaskReportPayload report) + { + try { + complete = true; + + final MSQStatusReport statusReport = report.getStatus(); + + if (statusReport.getStatus().isSuccess()) { + rowBuffer.put(Either.value(null)); + } else { + rowBuffer.put(Either.error(new MSQException(statusReport.getErrorReport().getFault()))); + } + } + catch (InterruptedException e) { + // Can't fix this by putting an error, because the rowBuffer isn't accepting new entries. + // Give up, allow controller.run() to fail. + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + } + + private void attach(final Closeable closeable) + { + synchronized (closer) { + closer.register(closeable); + } + } + + @Override + public void close() throws IOException + { + synchronized (closer) { + closer.close(); + } + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/sql/DartSqlClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/sql/DartSqlClient.java new file mode 100644 index 000000000000..447da229d05e --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/sql/DartSqlClient.java @@ -0,0 +1,42 @@ +/* + * 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.dart.controller.sql; + +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.msq.dart.controller.http.DartSqlResource; +import org.apache.druid.msq.dart.controller.http.GetQueriesResponse; + +import javax.servlet.http.HttpServletRequest; + +/** + * Client for the {@link DartSqlResource} resource. + */ +public interface DartSqlClient +{ + /** + * Get information about all currently-running queries on this server. + * + * @param selfOnly true if only queries from this server should be returned; false if queries from all servers + * should be returned + * + * @see DartSqlResource#doGetRunningQueries(String, HttpServletRequest) the server side + */ + ListenableFuture getRunningQueries(boolean selfOnly); +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/sql/DartSqlClientFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/sql/DartSqlClientFactory.java new file mode 100644 index 000000000000..879cabe6945f --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/sql/DartSqlClientFactory.java @@ -0,0 +1,30 @@ +/* + * 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.dart.controller.sql; + +import org.apache.druid.server.DruidNode; + +/** + * Generates {@link DartSqlClient} given a target Broker node. + */ +public interface DartSqlClientFactory +{ + DartSqlClient makeClient(DruidNode node); +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/sql/DartSqlClientFactoryImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/sql/DartSqlClientFactoryImpl.java new file mode 100644 index 000000000000..c2355a43e31a --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/sql/DartSqlClientFactoryImpl.java @@ -0,0 +1,64 @@ +/* + * 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.dart.controller.sql; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Inject; +import org.apache.druid.guice.annotations.EscalatedGlobal; +import org.apache.druid.guice.annotations.Json; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.msq.dart.controller.http.DartSqlResource; +import org.apache.druid.rpc.FixedServiceLocator; +import org.apache.druid.rpc.ServiceClient; +import org.apache.druid.rpc.ServiceClientFactory; +import org.apache.druid.rpc.ServiceLocation; +import org.apache.druid.rpc.StandardRetryPolicy; +import org.apache.druid.server.DruidNode; + +/** + * Production implementation of {@link DartSqlClientFactory}. + */ +public class DartSqlClientFactoryImpl implements DartSqlClientFactory +{ + private final ServiceClientFactory clientFactory; + private final ObjectMapper jsonMapper; + + @Inject + public DartSqlClientFactoryImpl( + @EscalatedGlobal final ServiceClientFactory clientFactory, + @Json final ObjectMapper jsonMapper + ) + { + this.clientFactory = clientFactory; + this.jsonMapper = jsonMapper; + } + + @Override + public DartSqlClient makeClient(DruidNode node) + { + final ServiceClient client = clientFactory.makeClient( + StringUtils.format("%s[dart-sql]", node.getHostAndPortToUse()), + new FixedServiceLocator(ServiceLocation.fromDruidNode(node).withBasePath(DartSqlResource.PATH)), + StandardRetryPolicy.noRetries() + ); + + return new DartSqlClientImpl(client, jsonMapper); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/sql/DartSqlClientImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/sql/DartSqlClientImpl.java new file mode 100644 index 000000000000..aebf7e4b90fa --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/sql/DartSqlClientImpl.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.msq.dart.controller.sql; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.java.util.common.jackson.JacksonUtils; +import org.apache.druid.java.util.http.client.response.BytesFullResponseHandler; +import org.apache.druid.msq.dart.controller.http.GetQueriesResponse; +import org.apache.druid.rpc.RequestBuilder; +import org.apache.druid.rpc.ServiceClient; +import org.jboss.netty.handler.codec.http.HttpMethod; + +/** + * Production implementation of {@link DartSqlClient}. + */ +public class DartSqlClientImpl implements DartSqlClient +{ + private final ServiceClient client; + private final ObjectMapper jsonMapper; + + public DartSqlClientImpl(final ServiceClient client, final ObjectMapper jsonMapper) + { + this.client = client; + this.jsonMapper = jsonMapper; + } + + @Override + public ListenableFuture getRunningQueries(final boolean selfOnly) + { + return FutureUtils.transform( + client.asyncRequest( + new RequestBuilder(HttpMethod.GET, selfOnly ? "/?selfOnly" : "/"), + new BytesFullResponseHandler() + ), + holder -> JacksonUtils.readValue(jsonMapper, holder.getContent(), GetQueriesResponse.class) + ); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/sql/DartSqlClients.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/sql/DartSqlClients.java new file mode 100644 index 000000000000..3fba872707a4 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/sql/DartSqlClients.java @@ -0,0 +1,118 @@ +/* + * 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.dart.controller.sql; + +import com.google.common.collect.ImmutableList; +import com.google.errorprone.annotations.concurrent.GuardedBy; +import com.google.inject.Inject; +import org.apache.druid.discovery.DiscoveryDruidNode; +import org.apache.druid.discovery.DruidNodeDiscovery; +import org.apache.druid.discovery.DruidNodeDiscoveryProvider; +import org.apache.druid.discovery.NodeRole; +import org.apache.druid.guice.ManageLifecycle; +import org.apache.druid.guice.annotations.Self; +import org.apache.druid.java.util.common.lifecycle.LifecycleStart; +import org.apache.druid.java.util.common.lifecycle.LifecycleStop; +import org.apache.druid.msq.dart.controller.http.DartSqlResource; +import org.apache.druid.server.DruidNode; + +import javax.servlet.http.HttpServletRequest; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Keeps {@link DartSqlClient} for all servers except ourselves. Currently the purpose of this is to power + * the "get all queries" API at {@link DartSqlResource#doGet(HttpServletRequest)}. + */ +@ManageLifecycle +public class DartSqlClients implements DruidNodeDiscovery.Listener +{ + @GuardedBy("servers") + private final Map clients = new HashMap<>(); + private final DruidNode selfNode; + private final DruidNodeDiscoveryProvider discoveryProvider; + private final DartSqlClientFactory clientFactory; + + private volatile DruidNodeDiscovery discovery; + + @Inject + public DartSqlClients( + @Self DruidNode selfNode, + DruidNodeDiscoveryProvider discoveryProvider, + DartSqlClientFactory clientFactory + ) + { + this.selfNode = selfNode; + this.discoveryProvider = discoveryProvider; + this.clientFactory = clientFactory; + } + + @LifecycleStart + public void start() + { + discovery = discoveryProvider.getForNodeRole(NodeRole.BROKER); + discovery.registerListener(this); + } + + public List getAllClients() + { + synchronized (clients) { + return ImmutableList.copyOf(clients.values()); + } + } + + @Override + public void nodesAdded(final Collection nodes) + { + synchronized (clients) { + for (final DiscoveryDruidNode node : nodes) { + final DruidNode druidNode = node.getDruidNode(); + if (!selfNode.equals(druidNode)) { + clients.computeIfAbsent(druidNode, clientFactory::makeClient); + } + } + } + } + + @Override + public void nodesRemoved(final Collection nodes) + { + synchronized (clients) { + for (final DiscoveryDruidNode node : nodes) { + clients.remove(node.getDruidNode()); + } + } + } + + @LifecycleStop + public void stop() + { + if (discovery != null) { + discovery.removeListener(this); + discovery = null; + } + + synchronized (clients) { + clients.clear(); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/sql/DartSqlEngine.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/sql/DartSqlEngine.java new file mode 100644 index 000000000000..e2818e017c95 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/controller/sql/DartSqlEngine.java @@ -0,0 +1,170 @@ +/* + * 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.dart.controller.sql; + +import com.google.common.collect.ImmutableList; +import org.apache.calcite.rel.RelRoot; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.msq.dart.controller.DartControllerRegistry; +import org.apache.druid.msq.exec.ControllerContext; +import org.apache.druid.msq.sql.MSQTaskSqlEngine; +import org.apache.druid.msq.dart.guice.DartControllerConfig; +import org.apache.druid.query.QueryContext; +import org.apache.druid.sql.calcite.planner.Calcites; +import org.apache.druid.sql.calcite.planner.PlannerContext; +import org.apache.druid.sql.calcite.run.EngineFeature; +import org.apache.druid.sql.calcite.run.QueryMaker; +import org.apache.druid.sql.calcite.run.SqlEngine; +import org.apache.druid.sql.calcite.run.SqlEngines; +import org.apache.druid.sql.destination.IngestDestination; + +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Semaphore; + +public class DartSqlEngine implements SqlEngine +{ + private static final String NAME = "msq-dart"; + public static final String CTX_DART_QUERY_ID = "dartQueryId"; + public static final String CTX_FULL_REPORT = "fullReport"; + public static final boolean CTX_FULL_REPORT_DEFAULT = false; + + private final ControllerContext controllerContext; + private final DartControllerRegistry controllerRegistry; + private final DartControllerConfig controllerConfig; + private final Semaphore controllerSemaphore; + private final ExecutorService controllerExecutor; + + public DartSqlEngine( + ControllerContext controllerContext, + DartControllerRegistry controllerRegistry, + DartControllerConfig controllerConfig, + Semaphore controllerSemaphore, + ExecutorService controllerExecutor + ) + { + this.controllerContext = controllerContext; + this.controllerRegistry = controllerRegistry; + this.controllerConfig = controllerConfig; + this.controllerSemaphore = controllerSemaphore; + this.controllerExecutor = controllerExecutor; + } + + @Override + public String name() + { + return NAME; + } + + @Override + public boolean featureAvailable(EngineFeature feature) + { + switch (feature) { + case CAN_SELECT: + case SCAN_ORDER_BY_NON_TIME: + case SCAN_NEEDS_SIGNATURE: + case WINDOW_FUNCTIONS: + case WINDOW_LEAF_OPERATOR: + case UNNEST: + return true; + + case CAN_INSERT: + case CAN_REPLACE: + case READ_EXTERNAL_DATA: + case ALLOW_BINDABLE_PLAN: + case ALLOW_BROADCAST_RIGHTY_JOIN: + case ALLOW_TOP_LEVEL_UNION_ALL: + case TIMESERIES_QUERY: + case TOPN_QUERY: + case TIME_BOUNDARY_QUERY: + case GROUPING_SETS: + case GROUPBY_IMPLICITLY_SORTS: + return false; + + default: + throw new IAE("Unrecognized feature: %s", feature); + } + } + + @Override + public void validateContext(Map queryContext) + { + SqlEngines.validateNoSpecialContextKeys(queryContext, MSQTaskSqlEngine.SYSTEM_CONTEXT_PARAMETERS); + } + + @Override + public RelDataType resultTypeForSelect( + RelDataTypeFactory typeFactory, + RelDataType validatedRowType, + Map queryContext + ) + { + if (QueryContext.of(queryContext).getBoolean(CTX_FULL_REPORT, CTX_FULL_REPORT_DEFAULT)) { + return typeFactory.createStructType( + ImmutableList.of( + Calcites.createSqlType(typeFactory, SqlTypeName.VARCHAR) + ), + ImmutableList.of(CTX_FULL_REPORT) + ); + } else { + return validatedRowType; + } + } + + @Override + public RelDataType resultTypeForInsert( + RelDataTypeFactory typeFactory, + RelDataType validatedRowType, + Map queryContext + ) + { + // Defensive, because we expect this method will not be called without the CAN_INSERT and CAN_REPLACE features. + throw DruidException.defensive("Cannot execute DML commands with engine[%s]", name()); + } + + @Override + public QueryMaker buildQueryMakerForSelect(RelRoot relRoot, PlannerContext plannerContext) + { + return new DartQueryMaker( + relRoot.fields, + controllerContext, + plannerContext, + controllerRegistry, + controllerConfig, + controllerSemaphore, + controllerExecutor + ); + } + + @Override + public QueryMaker buildQueryMakerForInsert( + IngestDestination destination, + RelRoot relRoot, + PlannerContext plannerContext + ) + { + // Defensive, because we expect this method will not be called without the CAN_INSERT and CAN_REPLACE features. + throw DruidException.defensive("Cannot execute DML commands with engine[%s]", name()); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/guice/DartControllerConfig.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/guice/DartControllerConfig.java new file mode 100644 index 000000000000..25094f44a79a --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/guice/DartControllerConfig.java @@ -0,0 +1,44 @@ +/* + * 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.dart.guice; + +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Runtime configuration for controllers (which run on Brokers). + */ +public class DartControllerConfig +{ + @JsonProperty("concurrentQueries") + private int concurrentQueries = 1; + + @JsonProperty("maxQueryReportSize") + private int maxQueryReportSize = 100_000_000; + + public int getConcurrentQueries() + { + return concurrentQueries; + } + + public int getMaxQueryReportSize() + { + return maxQueryReportSize; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/guice/DartControllerMemoryManagementModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/guice/DartControllerMemoryManagementModule.java new file mode 100644 index 000000000000..dfb2b2f8298a --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/guice/DartControllerMemoryManagementModule.java @@ -0,0 +1,64 @@ +/* + * 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.dart.guice; + +import com.google.inject.Binder; +import com.google.inject.Provides; +import org.apache.druid.discovery.NodeRole; +import org.apache.druid.guice.annotations.LoadScope; +import org.apache.druid.initialization.DruidModule; +import org.apache.druid.msq.exec.MemoryIntrospector; +import org.apache.druid.msq.exec.MemoryIntrospectorImpl; +import org.apache.druid.query.DruidProcessingConfig; +import org.apache.druid.utils.JvmUtils; + +/** + * Memory management module for Brokers. + */ +@LoadScope(roles = {NodeRole.BROKER_JSON_NAME}) +public class DartControllerMemoryManagementModule implements DruidModule +{ + /** + * Allocate up to 15% of memory for the MSQ framework. This accounts for additional overhead due to native queries, + * the segment timeline, and lookups (which aren't accounted for by our {@link MemoryIntrospector}). + */ + public static final double USABLE_MEMORY_FRACTION = 0.2; + + @Override + public void configure(Binder binder) + { + // Nothing to do. + } + + @Provides + public MemoryIntrospector createMemoryIntrospector( + final DruidProcessingConfig processingConfig, + final DartControllerConfig controllerConfig + ) + { + return new MemoryIntrospectorImpl( + JvmUtils.getRuntimeInfo().getMaxHeapSizeBytes(), + USABLE_MEMORY_FRACTION, + controllerConfig.getConcurrentQueries(), + processingConfig.getNumThreads(), + null + ); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/guice/DartControllerModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/guice/DartControllerModule.java new file mode 100644 index 000000000000..30950d9d4615 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/guice/DartControllerModule.java @@ -0,0 +1,159 @@ +/* + * 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.dart.guice; + +import com.google.inject.Binder; +import com.google.inject.Inject; +import com.google.inject.Module; +import com.google.inject.Provides; +import org.apache.druid.discovery.DruidNodeDiscoveryProvider; +import org.apache.druid.discovery.NodeRole; +import org.apache.druid.guice.Jerseys; +import org.apache.druid.guice.JsonConfigProvider; +import org.apache.druid.guice.LazySingleton; +import org.apache.druid.guice.LifecycleModule; +import org.apache.druid.guice.ManageLifecycle; +import org.apache.druid.guice.annotations.LoadScope; +import org.apache.druid.guice.annotations.Self; +import org.apache.druid.initialization.DruidModule; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.msq.dart.Dart; +import org.apache.druid.msq.dart.DartResourcePermissionMapper; +import org.apache.druid.msq.dart.controller.ControllerMessageListener; +import org.apache.druid.msq.dart.controller.ControllerServerId; +import org.apache.druid.msq.dart.controller.DartControllerContext; +import org.apache.druid.msq.dart.controller.DartControllerRegistry; +import org.apache.druid.msq.dart.controller.DartControllerRegistryImpl; +import org.apache.druid.msq.dart.controller.DartMessageRelayFactoryImpl; +import org.apache.druid.msq.dart.controller.DartMessageRelays; +import org.apache.druid.msq.dart.controller.http.DartSqlResource; +import org.apache.druid.msq.dart.controller.sql.DartSqlClientFactory; +import org.apache.druid.msq.dart.controller.sql.DartSqlClientFactoryImpl; +import org.apache.druid.msq.dart.controller.sql.DartSqlClients; +import org.apache.druid.msq.dart.controller.sql.DartSqlEngine; +import org.apache.druid.msq.exec.ControllerContext; +import org.apache.druid.msq.rpc.ResourcePermissionMapper; +import org.apache.druid.query.DefaultQueryConfig; +import org.apache.druid.server.DruidNode; +import org.apache.druid.sql.SqlStatementFactory; +import org.apache.druid.sql.SqlToolbox; + +import java.util.Properties; +import java.util.concurrent.Semaphore; + +/** + * Primary module for Brokers. Checks {@link DartModules#isDartEnabled(Properties)} before installing itself. + */ +@LoadScope(roles = NodeRole.BROKER_JSON_NAME) +public class DartControllerModule implements DruidModule +{ + @Inject + private Properties properties; + + @Override + public void configure(Binder binder) + { + if (DartModules.isDartEnabled(properties)) { + binder.install(new ActualModule()); + } + } + + public static class ActualModule implements Module + { + @Override + public void configure(Binder binder) + { + Jerseys.addResource(binder, DartSqlResource.class); + LifecycleModule.register(binder, DartSqlClients.class); + LifecycleModule.register(binder, DartMessageRelays.class); + JsonConfigProvider.bind(binder, DartModules.DART_PROPERTY_BASE + ".controller", DartControllerConfig.class); + JsonConfigProvider.bind(binder, DartModules.DART_PROPERTY_BASE + ".query", DefaultQueryConfig.class, Dart.class); + + binder.bind(ControllerContext.class) + .annotatedWith(Dart.class) + .to(DartControllerContext.class) + .in(LazySingleton.class); + + binder.bind(DartControllerRegistry.class) + .to(DartControllerRegistryImpl.class) + .in(LazySingleton.class); + + binder.bind(ControllerMessageListener.class) + .in(LazySingleton.class); + + binder.bind(DartMessageRelayFactoryImpl.class) + .in(LazySingleton.class); + + binder.bind(DartSqlClientFactory.class) + .to(DartSqlClientFactoryImpl.class) + .in(LazySingleton.class); + + binder.bind(ResourcePermissionMapper.class) + .annotatedWith(Dart.class) + .to(DartResourcePermissionMapper.class); + } + + @Provides + @LazySingleton + public ControllerServerId makeControllerId(final @Self DruidNode selfNode) + { + return new ControllerServerId(selfNode.getHostAndPortToUse(), DateTimes.nowUtc().getMillis()); + } + + @Provides + @Dart + @LazySingleton + public SqlStatementFactory makeSqlStatementFactory( + final DartSqlEngine engine, + final SqlToolbox toolbox + ) + { + return new SqlStatementFactory(toolbox.withEngine(engine)); + } + + @Provides + @ManageLifecycle + public DartMessageRelays makeMessageRelays( + final DruidNodeDiscoveryProvider discoveryProvider, + final DartMessageRelayFactoryImpl messageCollectorFactory + ) + { + return new DartMessageRelays(discoveryProvider, messageCollectorFactory); + } + + @Provides + @LazySingleton + public DartSqlEngine makeSqlEngine( + @Dart ControllerContext controllerContext, + DartControllerRegistry controllerRegistry, + DartControllerConfig controllerConfig + ) + { + return new DartSqlEngine( + controllerContext, + controllerRegistry, + controllerConfig, + new Semaphore(controllerConfig.getConcurrentQueries()), + Execs.multiThreaded(controllerConfig.getConcurrentQueries(), "dart-controller-%s") + ); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/guice/DartModules.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/guice/DartModules.java new file mode 100644 index 000000000000..a8e1a1b65e69 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/guice/DartModules.java @@ -0,0 +1,37 @@ +/* + * 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.dart.guice; + +import java.util.Properties; + +/** + * Common utilities for Dart Guice modules. + */ +public class DartModules +{ + public static final String DART_PROPERTY_BASE = "druid.msq.dart"; + public static final String DART_ENABLED_PROPERTY = DART_PROPERTY_BASE + ".enabled"; + public static final String DART_ENABLED_DEFAULT = String.valueOf(false); + + public static boolean isDartEnabled(final Properties properties) + { + return Boolean.parseBoolean(properties.getProperty(DART_ENABLED_PROPERTY, DART_ENABLED_DEFAULT)); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/guice/DartWorkerConfig.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/guice/DartWorkerConfig.java new file mode 100644 index 000000000000..f0cc1fd01025 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/guice/DartWorkerConfig.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.dart.guice; + +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.msq.exec.MemoryIntrospector; + +/** + * Runtime configuration for workers (which run on Historicals). + */ +public class DartWorkerConfig +{ + /** + * By default, allocate up to 35% of memory for the MSQ framework. This accounts for additional overhead due to + * native queries, and lookups (which aren't accounted for by the Dart {@link MemoryIntrospector}). + */ + private static final double DEFAULT_HEAP_FRACTION = 0.35; + + public static int AUTO = -1; + + @JsonProperty("concurrentQueries") + private int concurrentQueries = -1; + + @JsonProperty("heapFraction") + private double heapFraction = DEFAULT_HEAP_FRACTION; + + public int getConcurrentQueries() + { + return concurrentQueries; + } + + public double getHeapFraction() + { + return heapFraction; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/guice/DartWorkerMemoryManagementModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/guice/DartWorkerMemoryManagementModule.java new file mode 100644 index 000000000000..9d512ed5a16d --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/guice/DartWorkerMemoryManagementModule.java @@ -0,0 +1,100 @@ +/* + * 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.dart.guice; + +import com.google.inject.Binder; +import com.google.inject.Provides; +import org.apache.druid.collections.BlockingPool; +import org.apache.druid.discovery.NodeRole; +import org.apache.druid.error.DruidException; +import org.apache.druid.guice.LazySingleton; +import org.apache.druid.guice.annotations.LoadScope; +import org.apache.druid.guice.annotations.Merging; +import org.apache.druid.initialization.DruidModule; +import org.apache.druid.msq.dart.worker.DartProcessingBuffersProvider; +import org.apache.druid.msq.exec.MemoryIntrospector; +import org.apache.druid.msq.exec.MemoryIntrospectorImpl; +import org.apache.druid.msq.exec.ProcessingBuffersProvider; +import org.apache.druid.query.DruidProcessingConfig; +import org.apache.druid.utils.JvmUtils; + +import java.nio.ByteBuffer; + +/** + * Memory management module for Historicals. + */ +@LoadScope(roles = {NodeRole.HISTORICAL_JSON_NAME}) +public class DartWorkerMemoryManagementModule implements DruidModule +{ + @Override + public void configure(Binder binder) + { + // Nothing to do. + } + + @Provides + public MemoryIntrospector createMemoryIntrospector( + final DartWorkerConfig workerConfig, + final DruidProcessingConfig druidProcessingConfig + ) + { + return new MemoryIntrospectorImpl( + JvmUtils.getRuntimeInfo().getMaxHeapSizeBytes(), + workerConfig.getHeapFraction(), + computeConcurrentQueries(workerConfig, druidProcessingConfig), + druidProcessingConfig.getNumThreads(), + null + ); + } + + @Provides + @LazySingleton + public ProcessingBuffersProvider createProcessingBuffersProvider( + @Merging final BlockingPool mergeBufferPool, + final DruidProcessingConfig processingConfig + ) + { + return new DartProcessingBuffersProvider(mergeBufferPool, processingConfig.getNumThreads()); + } + + private static int computeConcurrentQueries( + final DartWorkerConfig workerConfig, + final DruidProcessingConfig processingConfig + ) + { + if (workerConfig.getConcurrentQueries() == DartWorkerConfig.AUTO) { + return processingConfig.getNumMergeBuffers(); + } else if (workerConfig.getConcurrentQueries() < 0) { + throw DruidException.forPersona(DruidException.Persona.OPERATOR) + .ofCategory(DruidException.Category.RUNTIME_FAILURE) + .build("concurrentQueries[%s] must be positive or -1", workerConfig.getConcurrentQueries()); + } else if (workerConfig.getConcurrentQueries() > processingConfig.getNumMergeBuffers()) { + throw DruidException.forPersona(DruidException.Persona.OPERATOR) + .ofCategory(DruidException.Category.RUNTIME_FAILURE) + .build( + "concurrentQueries[%s] must be less than numMergeBuffers[%s]", + workerConfig.getConcurrentQueries(), + processingConfig.getNumMergeBuffers() + ); + } else { + return workerConfig.getConcurrentQueries(); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/guice/DartWorkerModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/guice/DartWorkerModule.java new file mode 100644 index 000000000000..59b9ba214fb4 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/guice/DartWorkerModule.java @@ -0,0 +1,153 @@ +/* + * 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.dart.guice; + +import com.google.inject.Binder; +import com.google.inject.Inject; +import com.google.inject.Key; +import com.google.inject.Module; +import com.google.inject.Provides; +import org.apache.druid.discovery.DruidNodeDiscoveryProvider; +import org.apache.druid.discovery.NodeRole; +import org.apache.druid.guice.Jerseys; +import org.apache.druid.guice.JsonConfigProvider; +import org.apache.druid.guice.LazySingleton; +import org.apache.druid.guice.LifecycleModule; +import org.apache.druid.guice.ManageLifecycle; +import org.apache.druid.guice.ManageLifecycleAnnouncements; +import org.apache.druid.guice.annotations.LoadScope; +import org.apache.druid.guice.annotations.Self; +import org.apache.druid.initialization.DruidModule; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.messages.server.MessageRelayResource; +import org.apache.druid.messages.server.Outbox; +import org.apache.druid.messages.server.OutboxImpl; +import org.apache.druid.messages.server.ClientMonitor; +import org.apache.druid.msq.dart.Dart; +import org.apache.druid.msq.dart.DartResourcePermissionMapper; +import org.apache.druid.msq.dart.controller.messages.ControllerMessage; +import org.apache.druid.msq.dart.worker.DartDataSegmentProvider; +import org.apache.druid.msq.dart.worker.DartWorkerFactory; +import org.apache.druid.msq.dart.worker.DartWorkerFactoryImpl; +import org.apache.druid.msq.dart.worker.DartWorkerRunner; +import org.apache.druid.msq.dart.worker.http.DartWorkerResource; +import org.apache.druid.msq.exec.MemoryIntrospector; +import org.apache.druid.msq.querykit.DataSegmentProvider; +import org.apache.druid.msq.rpc.ResourcePermissionMapper; +import org.apache.druid.query.DruidProcessingConfig; +import org.apache.druid.server.DruidNode; +import org.apache.druid.server.security.AuthorizerMapper; + +import java.io.File; +import java.util.Properties; +import java.util.concurrent.ExecutorService; + +/** + * Primary module for workers. Checks {@link DartModules#isDartEnabled(Properties)} before installing itself. + */ +@LoadScope(roles = NodeRole.HISTORICAL_JSON_NAME) +public class DartWorkerModule implements DruidModule +{ + @Inject + private Properties properties; + + @Override + public void configure(Binder binder) + { + if (DartModules.isDartEnabled(properties)) { + binder.install(new ActualModule()); + } + } + + public static class ActualModule implements Module + { + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bind(binder, DartModules.DART_PROPERTY_BASE + ".worker", DartWorkerConfig.class); + Jerseys.addResource(binder, DartWorkerResource.class); + LifecycleModule.register(binder, DartWorkerRunner.class); + LifecycleModule.registerKey(binder, Key.get(ClientMonitor.class, Dart.class)); + + binder.bind(DartWorkerFactory.class) + .to(DartWorkerFactoryImpl.class) + .in(LazySingleton.class); + + binder.bind(DataSegmentProvider.class) + .annotatedWith(Dart.class) + .to(DartDataSegmentProvider.class) + .in(LazySingleton.class); + + binder.bind(ResourcePermissionMapper.class) + .annotatedWith(Dart.class) + .to(DartResourcePermissionMapper.class); + } + + @Provides + @ManageLifecycle + public DartWorkerRunner createWorkerRunner( + @Self final DruidNode selfNode, + final DartWorkerFactory workerFactory, + final DruidNodeDiscoveryProvider discoveryProvider, + final DruidProcessingConfig processingConfig, + @Dart final ResourcePermissionMapper permissionMapper, + final MemoryIntrospector memoryIntrospector, + final AuthorizerMapper authorizerMapper + ) + { + final ExecutorService exec = Execs.multiThreaded(memoryIntrospector.numTasksInJvm(), "dart–worker-%s"); + final File baseTempDir = + new File(processingConfig.getTmpDir(), StringUtils.format("dart_%s", selfNode.getPortToUse())); + return new DartWorkerRunner( + workerFactory, + exec, + discoveryProvider, + permissionMapper, + authorizerMapper, + baseTempDir + ); + } + + @Provides + @Dart + public ClientMonitor createControllerMonitor( + final DruidNodeDiscoveryProvider discoveryProvider, + final Outbox outbox + ) + { + return new ClientMonitor(discoveryProvider, outbox, NodeRole.BROKER); + } + + /** + * Create an {@link Outbox}. + * + * This is {@link ManageLifecycleAnnouncements} scoped so {@link OutboxImpl#stop()} gets called before attempting + * to shut down the Jetty server. If this doesn't happen, then server shutdown is delayed by however long it takes + * any currently-in-flight {@link MessageRelayResource#httpGetMessagesFromOutbox} to resolve. + */ + @Provides + @ManageLifecycleAnnouncements + public Outbox createOutbox() + { + return new OutboxImpl<>(); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartControllerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartControllerClient.java new file mode 100644 index 000000000000..2e9174f6b406 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartControllerClient.java @@ -0,0 +1,137 @@ +/* + * 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.dart.worker; + +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.error.DruidException; +import org.apache.druid.messages.server.Outbox; +import org.apache.druid.msq.counters.CounterSnapshotsTree; +import org.apache.druid.msq.dart.controller.messages.ControllerMessage; +import org.apache.druid.msq.dart.controller.messages.DoneReadingInput; +import org.apache.druid.msq.dart.controller.messages.PartialKeyStatistics; +import org.apache.druid.msq.dart.controller.messages.ResultsComplete; +import org.apache.druid.msq.dart.controller.messages.WorkerError; +import org.apache.druid.msq.dart.controller.messages.WorkerWarning; +import org.apache.druid.msq.exec.ControllerClient; +import org.apache.druid.msq.indexing.error.MSQErrorReport; +import org.apache.druid.msq.kernel.StageId; +import org.apache.druid.msq.statistics.PartialKeyStatisticsInformation; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.List; + +/** + * Implementation of {@link ControllerClient} that uses an {@link Outbox} to send {@link ControllerMessage} + * to a controller. + */ +public class DartControllerClient implements ControllerClient +{ + private final Outbox outbox; + private final String queryId; + private final String controllerHost; + + public DartControllerClient( + final Outbox outbox, + final String queryId, + final String controllerHost + ) + { + this.outbox = outbox; + this.queryId = queryId; + this.controllerHost = controllerHost; + } + + @Override + public void postPartialKeyStatistics( + final StageId stageId, + final int workerNumber, + final PartialKeyStatisticsInformation partialKeyStatisticsInformation + ) + { + validateStage(stageId); + sendMessage(new PartialKeyStatistics(stageId, workerNumber, partialKeyStatisticsInformation)); + } + + @Override + public void postDoneReadingInput(StageId stageId, int workerNumber) + { + validateStage(stageId); + sendMessage(new DoneReadingInput(stageId, workerNumber)); + } + + @Override + public void postResultsComplete(StageId stageId, int workerNumber, @Nullable Object resultObject) + { + validateStage(stageId); + sendMessage(new ResultsComplete(stageId, workerNumber, resultObject)); + } + + @Override + public void postWorkerError(MSQErrorReport errorWrapper) + { + sendMessage(new WorkerError(queryId, errorWrapper)); + } + + @Override + public void postWorkerWarning(List errorWrappers) + { + sendMessage(new WorkerWarning(queryId, errorWrappers)); + } + + @Override + public void postCounters(String workerId, CounterSnapshotsTree snapshotsTree) + { + // Do nothing. Live counters are not sent to the controller in this mode. + } + + @Override + public List getWorkerIds() throws IOException + { + // Workers are set in advance through the WorkOrder, so this method isn't used. + throw new UnsupportedOperationException(); + } + + @Override + public void close() + { + // Nothing to close. + } + + private void sendMessage(final ControllerMessage message) + { + FutureUtils.getUnchecked(outbox.sendMessage(controllerHost, message), true); + } + + /** + * Validate that a {@link StageId} has the expected query ID. + */ + private void validateStage(final StageId stageId) + { + if (!stageId.getQueryId().equals(queryId)) { + throw DruidException.defensive( + "Expected queryId[%s] but got queryId[%s], stageNumber[%s]", + queryId, + stageId.getQueryId(), + stageId.getStageNumber() + ); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartDataSegmentProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartDataSegmentProvider.java new file mode 100644 index 000000000000..0e8a38af90a3 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartDataSegmentProvider.java @@ -0,0 +1,111 @@ +/* + * 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.dart.worker; + +import com.google.inject.Inject; +import org.apache.druid.collections.ReferenceCountingResourceHolder; +import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.msq.counters.ChannelCounters; +import org.apache.druid.msq.querykit.DataSegmentProvider; +import org.apache.druid.query.TableDataSource; +import org.apache.druid.segment.CompleteSegment; +import org.apache.druid.segment.PhysicalSegmentInspector; +import org.apache.druid.segment.ReferenceCountingSegment; +import org.apache.druid.server.SegmentManager; +import org.apache.druid.timeline.SegmentId; +import org.apache.druid.timeline.VersionedIntervalTimeline; +import org.apache.druid.timeline.partition.PartitionChunk; + +import java.io.Closeable; +import java.util.Optional; +import java.util.function.Supplier; + +/** + * Implementation of {@link DataSegmentProvider} that uses locally-cached segments from a {@link SegmentManager}. + */ +public class DartDataSegmentProvider implements DataSegmentProvider +{ + private final SegmentManager segmentManager; + + @Inject + public DartDataSegmentProvider(SegmentManager segmentManager) + { + this.segmentManager = segmentManager; + } + + @Override + public Supplier> fetchSegment( + SegmentId segmentId, + ChannelCounters channelCounters, + boolean isReindex + ) + { + if (isReindex) { + throw DruidException.defensive("Got isReindex[%s], expected false", isReindex); + } + + return () -> { + final Optional> timeline = + segmentManager.getTimeline(new TableDataSource(segmentId.getDataSource()).getAnalysis()); + + if (!timeline.isPresent()) { + throw segmentNotFound(segmentId); + } + + final PartitionChunk chunk = + timeline.get().findChunk( + segmentId.getInterval(), + segmentId.getVersion(), + segmentId.getPartitionNum() + ); + + if (chunk == null) { + throw segmentNotFound(segmentId); + } + + final ReferenceCountingSegment segment = chunk.getObject(); + final Optional closeable = segment.acquireReferences(); + if (!closeable.isPresent()) { + // Segment has disappeared before we could acquire a reference to it. + throw segmentNotFound(segmentId); + } + + final Closer closer = Closer.create(); + closer.register(closeable.get()); + closer.register(() -> { + final PhysicalSegmentInspector inspector = segment.as(PhysicalSegmentInspector.class); + channelCounters.addFile(inspector != null ? inspector.getNumRows() : 0, 0); + }); + return new ReferenceCountingResourceHolder<>(new CompleteSegment(null, segment), closer); + }; + } + + /** + * Error to throw when a segment that was requested is not found. This can happen due to segment moves, etc. + */ + private static DruidException segmentNotFound(final SegmentId segmentId) + { + return DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.RUNTIME_FAILURE) + .build("Segment[%s] not found on this server. Please retry your query.", segmentId); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartFrameContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartFrameContext.java new file mode 100644 index 000000000000..ff7d9fdc4e9f --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartFrameContext.java @@ -0,0 +1,178 @@ +/* + * 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.dart.worker; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.msq.exec.DataServerQueryHandlerFactory; +import org.apache.druid.msq.exec.ProcessingBuffers; +import org.apache.druid.msq.exec.WorkerContext; +import org.apache.druid.msq.exec.WorkerMemoryParameters; +import org.apache.druid.msq.exec.WorkerStorageParameters; +import org.apache.druid.msq.kernel.FrameContext; +import org.apache.druid.msq.kernel.StageId; +import org.apache.druid.msq.querykit.DataSegmentProvider; +import org.apache.druid.query.groupby.GroupingEngine; +import org.apache.druid.segment.IndexIO; +import org.apache.druid.segment.IndexMergerV9; +import org.apache.druid.segment.SegmentWrangler; +import org.apache.druid.segment.incremental.NoopRowIngestionMeters; +import org.apache.druid.segment.incremental.RowIngestionMeters; +import org.apache.druid.segment.loading.DataSegmentPusher; + +import javax.annotation.Nullable; +import java.io.File; + +/** + * Dart implementation of {@link FrameContext}. + */ +public class DartFrameContext implements FrameContext +{ + private final StageId stageId; + private final SegmentWrangler segmentWrangler; + private final GroupingEngine groupingEngine; + private final DataSegmentProvider dataSegmentProvider; + private final WorkerContext workerContext; + @Nullable + private final ResourceHolder processingBuffers; + private final WorkerMemoryParameters memoryParameters; + private final WorkerStorageParameters storageParameters; + + public DartFrameContext( + final StageId stageId, + final WorkerContext workerContext, + final SegmentWrangler segmentWrangler, + final GroupingEngine groupingEngine, + final DataSegmentProvider dataSegmentProvider, + @Nullable ResourceHolder processingBuffers, + final WorkerMemoryParameters memoryParameters, + final WorkerStorageParameters storageParameters + ) + { + this.stageId = stageId; + this.segmentWrangler = segmentWrangler; + this.groupingEngine = groupingEngine; + this.dataSegmentProvider = dataSegmentProvider; + this.workerContext = workerContext; + this.processingBuffers = processingBuffers; + this.memoryParameters = memoryParameters; + this.storageParameters = storageParameters; + } + + @Override + public SegmentWrangler segmentWrangler() + { + return segmentWrangler; + } + + @Override + public GroupingEngine groupingEngine() + { + return groupingEngine; + } + + @Override + public RowIngestionMeters rowIngestionMeters() + { + return new NoopRowIngestionMeters(); + } + + @Override + public DataSegmentProvider dataSegmentProvider() + { + return dataSegmentProvider; + } + + @Override + public File tempDir() + { + return new File(workerContext.tempDir(), stageId.toString()); + } + + @Override + public ObjectMapper jsonMapper() + { + return workerContext.jsonMapper(); + } + + @Override + public IndexIO indexIO() + { + throw new UnsupportedOperationException(); + } + + @Override + public File persistDir() + { + return new File(tempDir(), "persist"); + } + + @Override + public DataSegmentPusher segmentPusher() + { + throw DruidException.defensive("Ingestion not implemented"); + } + + @Override + public IndexMergerV9 indexMerger() + { + throw DruidException.defensive("Ingestion not implemented"); + } + + @Override + public ProcessingBuffers processingBuffers() + { + if (processingBuffers != null) { + return processingBuffers.get(); + } else { + throw new ISE("No processing buffers"); + } + } + + @Override + public WorkerMemoryParameters memoryParameters() + { + return memoryParameters; + } + + @Override + public WorkerStorageParameters storageParameters() + { + return storageParameters; + } + + @Override + public DataServerQueryHandlerFactory dataServerQueryHandlerFactory() + { + // We don't query data servers. This factory won't actually be used, because Dart doesn't allow segmentSource to be + // overridden; it always uses SegmentSource.NONE. (If it is called, some wires got crossed somewhere.) + return null; + } + + @Override + public void close() + { + if (processingBuffers != null) { + processingBuffers.close(); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartProcessingBuffersProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartProcessingBuffersProvider.java new file mode 100644 index 000000000000..02e933bcbace --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartProcessingBuffersProvider.java @@ -0,0 +1,94 @@ +/* + * 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.dart.worker; + +import org.apache.druid.collections.BlockingPool; +import org.apache.druid.collections.QueueNonBlockingPool; +import org.apache.druid.collections.ReferenceCountingResourceHolder; +import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.error.DruidException; +import org.apache.druid.frame.processor.Bouncer; +import org.apache.druid.msq.exec.ProcessingBuffers; +import org.apache.druid.msq.exec.ProcessingBuffersProvider; +import org.apache.druid.msq.exec.ProcessingBuffersSet; +import org.apache.druid.utils.CloseableUtils; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; + +/** + * Production implementation of {@link ProcessingBuffersProvider} that uses the merge buffer pool. Each call + * to {@link #acquire(int)} acquires one merge buffer and slices it up. + */ +public class DartProcessingBuffersProvider implements ProcessingBuffersProvider +{ + private final BlockingPool mergeBufferPool; + private final int processingThreads; + + public DartProcessingBuffersProvider(BlockingPool mergeBufferPool, int processingThreads) + { + this.mergeBufferPool = mergeBufferPool; + this.processingThreads = processingThreads; + } + + @Override + public ResourceHolder acquire(final int poolSize) + { + if (poolSize == 0) { + return new ReferenceCountingResourceHolder<>(ProcessingBuffersSet.EMPTY, () -> {}); + } + + final List> batch = mergeBufferPool.takeBatch(1, 0); + if (batch.isEmpty()) { + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.RUNTIME_FAILURE) + .build("No merge buffers available, cannot execute query"); + } + + final ReferenceCountingResourceHolder bufferHolder = batch.get(0); + try { + final ByteBuffer buffer = bufferHolder.get().duplicate(); + final List pool = new ArrayList<>(poolSize); + + for (int i = 0; i < poolSize; i++) { + final int sliceSize = buffer.capacity() / poolSize / processingThreads; + final BlockingQueue queue = new ArrayBlockingQueue<>(processingThreads); + for (int j = 0; j < processingThreads; j++) { + final int sliceNum = i * processingThreads + j; + buffer.position(sliceSize * sliceNum).limit(sliceSize * (sliceNum + 1)); + queue.add(buffer.slice()); + } + final ProcessingBuffers buffers = new ProcessingBuffers( + new QueueNonBlockingPool<>(queue), + new Bouncer(processingThreads) + ); + pool.add(buffers); + } + + return new ReferenceCountingResourceHolder<>(new ProcessingBuffersSet(pool), bufferHolder); + } + catch (Throwable e) { + throw CloseableUtils.closeAndWrapInCatch(e, bufferHolder); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartWorkerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartWorkerClient.java new file mode 100644 index 000000000000..f57db986a18d --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartWorkerClient.java @@ -0,0 +1,152 @@ +/* + * 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.dart.worker; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.errorprone.annotations.concurrent.GuardedBy; +import org.apache.druid.java.util.http.client.response.HttpResponseHandler; +import org.apache.druid.msq.dart.controller.ControllerServerId; +import org.apache.druid.msq.dart.controller.DartWorkerManager; +import org.apache.druid.msq.dart.worker.http.DartWorkerResource; +import org.apache.druid.msq.exec.WorkerClient; +import org.apache.druid.msq.rpc.BaseWorkerClientImpl; +import org.apache.druid.rpc.FixedServiceLocator; +import org.apache.druid.rpc.IgnoreHttpResponseHandler; +import org.apache.druid.rpc.RequestBuilder; +import org.apache.druid.rpc.ServiceClient; +import org.apache.druid.rpc.ServiceClientFactory; +import org.apache.druid.rpc.ServiceLocation; +import org.apache.druid.rpc.ServiceRetryPolicy; +import org.jboss.netty.handler.codec.http.HttpMethod; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.net.URI; +import java.util.HashMap; +import java.util.Map; + +/** + * Dart implementation of {@link WorkerClient}. Uses the same {@link BaseWorkerClientImpl} as the task-based engine. + */ +public class DartWorkerClient extends BaseWorkerClientImpl +{ + private final ServiceClientFactory clientFactory; + + @Nullable + private final ControllerServerId controllerServerId; + + @GuardedBy("clientMap") + private final Map clientMap = new HashMap<>(); + + /** + * Create a worker client. + * + * @param clientFactory service client factor + * @param smileMapper Smile object mapper + * @param controllerServerId Controller ID (see {@link DartWorkerResource#HEADER_CONTROLLER_ID}) if this is a + * controller-to-worker client. Null if this is a worker-to-worker client. + */ + public DartWorkerClient( + final ServiceClientFactory clientFactory, + final ObjectMapper smileMapper, + @Nullable final ControllerServerId controllerServerId + ) + { + super(smileMapper, SmileMediaTypes.APPLICATION_JACKSON_SMILE); + this.clientFactory = clientFactory; + this.controllerServerId = controllerServerId; + } + + @Override + protected ServiceClient getClient(String workerId) + { + synchronized (clientMap) { + return clientMap.computeIfAbsent( + workerId, + id -> { + final URI uri = WorkerId.fromString(id).toUri(); + final ServiceLocation location = ServiceLocation.fromUri(uri); + final FixedServiceLocator locator = new FixedServiceLocator(location); + final ServiceClient client = clientFactory.makeClient(id, locator, DartWorkerRetryPolicy.INSTANCE); + + if (controllerServerId != null) { + return new ControllerDecoratedClient(client, controllerServerId); + } else { + return client; + } + } + ); + } + } + + @Override + public void close() throws IOException + { + synchronized (clientMap) { + clientMap.clear(); + } + } + + /** + * Stops a worker. Dart-only API, used by the {@link DartWorkerManager#toString()}. + */ + public ListenableFuture stopWorker(String workerId) + { + return getClient(workerId).asyncRequest( + new RequestBuilder(HttpMethod.POST, "/stop"), + IgnoreHttpResponseHandler.INSTANCE + ); + } + + /** + * Service client that adds the {@link DartWorkerResource#HEADER_CONTROLLER_ID} header. + */ + private static class ControllerDecoratedClient implements ServiceClient + { + private final ServiceClient delegate; + private final ControllerServerId controllerServerId; + + ControllerDecoratedClient(final ServiceClient delegate, final ControllerServerId controllerServerId) + { + this.delegate = delegate; + this.controllerServerId = controllerServerId; + } + + @Override + public ListenableFuture asyncRequest( + final RequestBuilder requestBuilder, + final HttpResponseHandler handler + ) + { + return delegate.asyncRequest( + requestBuilder.header(DartWorkerResource.HEADER_CONTROLLER_ID, controllerServerId.toString()), + handler + ); + } + + @Override + public ServiceClient withRetryPolicy(final ServiceRetryPolicy retryPolicy) + { + return new ControllerDecoratedClient(delegate.withRetryPolicy(retryPolicy), controllerServerId); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartWorkerContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartWorkerContext.java new file mode 100644 index 000000000000..642b8471472e --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartWorkerContext.java @@ -0,0 +1,246 @@ +/* + * 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.dart.worker; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import com.google.errorprone.annotations.concurrent.GuardedBy; +import com.google.inject.Injector; +import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.messages.server.Outbox; +import org.apache.druid.msq.exec.ControllerClient; +import org.apache.druid.msq.exec.DataServerQueryHandlerFactory; +import org.apache.druid.msq.exec.MemoryIntrospector; +import org.apache.druid.msq.exec.ProcessingBuffersProvider; +import org.apache.druid.msq.exec.ProcessingBuffersSet; +import org.apache.druid.msq.exec.Worker; +import org.apache.druid.msq.exec.WorkerClient; +import org.apache.druid.msq.exec.WorkerContext; +import org.apache.druid.msq.exec.WorkerMemoryParameters; +import org.apache.druid.msq.exec.WorkerStorageParameters; +import org.apache.druid.msq.kernel.FrameContext; +import org.apache.druid.msq.kernel.WorkOrder; +import org.apache.druid.msq.querykit.DataSegmentProvider; +import org.apache.druid.msq.util.MultiStageQueryContext; +import org.apache.druid.msq.dart.controller.messages.ControllerMessage; +import org.apache.druid.query.DruidProcessingConfig; +import org.apache.druid.query.QueryContext; +import org.apache.druid.query.groupby.GroupingEngine; +import org.apache.druid.rpc.ServiceClientFactory; +import org.apache.druid.segment.SegmentWrangler; +import org.apache.druid.server.DruidNode; + +import java.io.File; + +/** + * Dart implementation of {@link WorkerContext}. + */ +public class DartWorkerContext implements WorkerContext +{ + private final String queryId; + private final String controllerHost; + private final String workerId; + private final DruidNode selfNode; + private final ObjectMapper jsonMapper; + private final ObjectMapper smileMapper; + private final Injector injector; + private final ServiceClientFactory serviceClientFactory; + private final DruidProcessingConfig processingConfig; + private final SegmentWrangler segmentWrangler; + private final GroupingEngine groupingEngine; + private final DataSegmentProvider dataSegmentProvider; + private final MemoryIntrospector memoryIntrospector; + private final ProcessingBuffersProvider processingBuffersProvider; + private final Outbox outbox; + private final File tempDir; + private final QueryContext queryContext; + + @GuardedBy("this") + private volatile ResourceHolder processingBuffersSet; + + DartWorkerContext( + final String queryId, + final String controllerHost, + final String workerId, + final DruidNode selfNode, + final ObjectMapper jsonMapper, + final ObjectMapper smileMapper, + final Injector injector, + final ServiceClientFactory serviceClientFactory, + final DruidProcessingConfig processingConfig, + final SegmentWrangler segmentWrangler, + final GroupingEngine groupingEngine, + final DataSegmentProvider dataSegmentProvider, + final MemoryIntrospector memoryIntrospector, + final ProcessingBuffersProvider processingBuffersProvider, + final Outbox outbox, + final File tempDir, + final QueryContext queryContext + ) + { + this.queryId = queryId; + this.controllerHost = controllerHost; + this.workerId = workerId; + this.selfNode = selfNode; + this.jsonMapper = jsonMapper; + this.smileMapper = smileMapper; + this.injector = injector; + this.serviceClientFactory = serviceClientFactory; + this.processingConfig = processingConfig; + this.segmentWrangler = segmentWrangler; + this.groupingEngine = groupingEngine; + this.dataSegmentProvider = dataSegmentProvider; + this.memoryIntrospector = memoryIntrospector; + this.processingBuffersProvider = processingBuffersProvider; + this.outbox = outbox; + this.tempDir = tempDir; + this.queryContext = Preconditions.checkNotNull(queryContext, "queryContext"); + } + + @Override + public String queryId() + { + return queryId; + } + + @Override + public String workerId() + { + return workerId; + } + + @Override + public ObjectMapper jsonMapper() + { + return jsonMapper; + } + + @Override + public Injector injector() + { + return injector; + } + + @Override + public void registerWorker(Worker worker, Closer closer) + { + closer.register(() -> { + synchronized (this) { + if (processingBuffersSet != null) { + processingBuffersSet.close(); + processingBuffersSet = null; + } + } + }); + } + + @Override + public int maxConcurrentStages() + { + final int retVal = MultiStageQueryContext.getMaxConcurrentStagesWithDefault(queryContext, -1); + if (retVal <= 0) { + throw new IAE("Illegal maxConcurrentStages[%s]", retVal); + } + return retVal; + } + + @Override + public ControllerClient makeControllerClient() + { + return new DartControllerClient(outbox, queryId, controllerHost); + } + + @Override + public WorkerClient makeWorkerClient() + { + return new DartWorkerClient(serviceClientFactory, smileMapper, null); + } + + @Override + public File tempDir() + { + return tempDir; + } + + @Override + public FrameContext frameContext(WorkOrder workOrder) + { + if (processingBuffersSet == null) { + synchronized (this) { + if (processingBuffersSet == null) { + processingBuffersSet = processingBuffersProvider.acquire( + workOrder.getQueryDefinition(), + maxConcurrentStages() + ); + } + } + } + + final WorkerMemoryParameters memoryParameters = + WorkerMemoryParameters.createProductionInstance( + workOrder, + memoryIntrospector, + maxConcurrentStages() + ); + + final WorkerStorageParameters storageParameters = WorkerStorageParameters.createInstance(-1, false); + + return new DartFrameContext( + workOrder.getStageDefinition().getId(), + this, + segmentWrangler, + groupingEngine, + dataSegmentProvider, + processingBuffersSet.get().acquireForStage(workOrder.getStageDefinition()), + memoryParameters, + storageParameters + ); + } + + @Override + public int threadCount() + { + return processingConfig.getNumThreads(); + } + + @Override + public DataServerQueryHandlerFactory dataServerQueryHandlerFactory() + { + // We don't query data servers. Return null so this factory is ignored when the main worker code tries + // to close it. + return null; + } + + @Override + public boolean includeAllCounters() + { + // The context parameter "includeAllCounters" is meant to assist with backwards compatibility for versions prior + // to Druid 31. Dart didn't exist prior to Druid 31, so there is no need for it here. Always emit all counters. + return true; + } + + @Override + public DruidNode selfNode() + { + return selfNode; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartWorkerFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartWorkerFactory.java new file mode 100644 index 000000000000..429579b2195e --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartWorkerFactory.java @@ -0,0 +1,33 @@ +/* + * 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.dart.worker; + +import org.apache.druid.msq.exec.Worker; +import org.apache.druid.query.QueryContext; + +import java.io.File; + +/** + * Used by {@link DartWorkerRunner} to create new {@link Worker} instances. + */ +public interface DartWorkerFactory +{ + Worker build(String queryId, String controllerHost, File tempDir, QueryContext context); +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartWorkerFactoryImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartWorkerFactoryImpl.java new file mode 100644 index 000000000000..eee682e257e8 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartWorkerFactoryImpl.java @@ -0,0 +1,143 @@ +/* + * 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.dart.worker; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Inject; +import com.google.inject.Injector; +import org.apache.druid.guice.annotations.EscalatedGlobal; +import org.apache.druid.guice.annotations.Json; +import org.apache.druid.guice.annotations.Self; +import org.apache.druid.guice.annotations.Smile; +import org.apache.druid.messages.server.Outbox; +import org.apache.druid.msq.dart.Dart; +import org.apache.druid.msq.dart.controller.messages.ControllerMessage; +import org.apache.druid.msq.dart.worker.http.DartWorkerResource; +import org.apache.druid.msq.exec.MemoryIntrospector; +import org.apache.druid.msq.exec.ProcessingBuffersProvider; +import org.apache.druid.msq.exec.Worker; +import org.apache.druid.msq.exec.WorkerContext; +import org.apache.druid.msq.exec.WorkerImpl; +import org.apache.druid.msq.querykit.DataSegmentProvider; +import org.apache.druid.query.DruidProcessingConfig; +import org.apache.druid.query.QueryContext; +import org.apache.druid.query.groupby.GroupingEngine; +import org.apache.druid.rpc.ServiceClientFactory; +import org.apache.druid.segment.SegmentWrangler; +import org.apache.druid.server.DruidNode; + +import java.io.File; +import java.net.URI; +import java.net.URISyntaxException; + +/** + * Production implementation of {@link DartWorkerFactory}. + */ +public class DartWorkerFactoryImpl implements DartWorkerFactory +{ + private final String id; + private final DruidNode selfNode; + private final ObjectMapper jsonMapper; + private final ObjectMapper smileMapper; + private final Injector injector; + private final ServiceClientFactory serviceClientFactory; + private final DruidProcessingConfig processingConfig; + private final SegmentWrangler segmentWrangler; + private final GroupingEngine groupingEngine; + private final DataSegmentProvider dataSegmentProvider; + private final MemoryIntrospector memoryIntrospector; + private final ProcessingBuffersProvider processingBuffersProvider; + private final Outbox outbox; + + @Inject + public DartWorkerFactoryImpl( + @Self DruidNode selfNode, + @Json ObjectMapper jsonMapper, + @Smile ObjectMapper smileMapper, + Injector injector, + @EscalatedGlobal ServiceClientFactory serviceClientFactory, + DruidProcessingConfig processingConfig, + SegmentWrangler segmentWrangler, + GroupingEngine groupingEngine, + @Dart DataSegmentProvider dataSegmentProvider, + MemoryIntrospector memoryIntrospector, + ProcessingBuffersProvider processingBuffersProvider, + Outbox outbox + ) + { + this.id = makeWorkerId(selfNode); + this.selfNode = selfNode; + this.jsonMapper = jsonMapper; + this.smileMapper = smileMapper; + this.injector = injector; + this.serviceClientFactory = serviceClientFactory; + this.processingConfig = processingConfig; + this.segmentWrangler = segmentWrangler; + this.groupingEngine = groupingEngine; + this.dataSegmentProvider = dataSegmentProvider; + this.memoryIntrospector = memoryIntrospector; + this.processingBuffersProvider = processingBuffersProvider; + this.outbox = outbox; + } + + @Override + public Worker build(String queryId, String controllerHost, File tempDir, QueryContext queryContext) + { + final WorkerContext workerContext = new DartWorkerContext( + queryId, + controllerHost, + id, + selfNode, + jsonMapper, + smileMapper, + injector, + serviceClientFactory, + processingConfig, + segmentWrangler, + groupingEngine, + dataSegmentProvider, + memoryIntrospector, + processingBuffersProvider, + outbox, + tempDir, + queryContext + ); + + return new WorkerImpl(null, workerContext); + } + + private static String makeWorkerId(final DruidNode selfNode) + { + try { + return new URI( + selfNode.getServiceScheme(), + null, + selfNode.getHost(), + selfNode.getPortToUse(), + DartWorkerResource.PATH, + null, + null + ).toString(); + } + catch (URISyntaxException e) { + throw new RuntimeException(e); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartWorkerRetryPolicy.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartWorkerRetryPolicy.java new file mode 100644 index 000000000000..00a5009fc131 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartWorkerRetryPolicy.java @@ -0,0 +1,94 @@ +/* + * 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.dart.worker; + +import org.apache.druid.messages.client.MessageListener; +import org.apache.druid.msq.dart.worker.http.DartWorkerResource; +import org.apache.druid.rpc.ServiceRetryPolicy; +import org.apache.druid.rpc.StandardRetryPolicy; +import org.apache.druid.server.DruidNode; +import org.jboss.netty.handler.codec.http.HttpResponse; + +/** + * Retry policy for {@link DartWorkerClient}. This is a {@link StandardRetryPolicy#unlimitedWithoutRetryLogging()} + * with an adjustment to retry 404, which can happen when the worker we're trying to contact hasn't been initialized + * yet. + */ +public class DartWorkerRetryPolicy implements ServiceRetryPolicy +{ + public static final DartWorkerRetryPolicy INSTANCE = new DartWorkerRetryPolicy(); + + /** + * Must be an unlimited policy to avoid zombie workers; see note in {@link DartWorkerResource#httpPostStopWorker}. + * + * If the worker we're trying to contact has gone, we'll notice that through either + * {@link MessageListener#serverRemoved(DruidNode)} (if we're a controller) or by receiving a stop command from + * a controller (if we're a worker). + */ + private static final ServiceRetryPolicy DELEGATE = StandardRetryPolicy.unlimitedWithoutRetryLogging(); + + private DartWorkerRetryPolicy() + { + // Singleton. + } + + @Override + public long maxAttempts() + { + return DELEGATE.maxAttempts(); + } + + @Override + public long minWaitMillis() + { + return DELEGATE.minWaitMillis(); + } + + @Override + public long maxWaitMillis() + { + return DELEGATE.maxWaitMillis(); + } + + @Override + public boolean retryHttpResponse(HttpResponse response) + { + // Retry 404, which can happen when the worker we're trying to contact hasn't been initialized yet. + return response.getStatus().getCode() == 404 || DELEGATE.retryHttpResponse(response); + } + + @Override + public boolean retryThrowable(Throwable t) + { + return DELEGATE.retryThrowable(t); + } + + @Override + public boolean retryLoggable() + { + return DELEGATE.retryLoggable(); + } + + @Override + public boolean retryNotAvailable() + { + return DELEGATE.retryNotAvailable(); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartWorkerRunner.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartWorkerRunner.java new file mode 100644 index 000000000000..5c5fe7bf34b6 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/DartWorkerRunner.java @@ -0,0 +1,333 @@ +/* + * 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.dart.worker; + +import com.google.errorprone.annotations.concurrent.GuardedBy; +import org.apache.druid.discovery.DiscoveryDruidNode; +import org.apache.druid.discovery.DruidNodeDiscovery; +import org.apache.druid.discovery.DruidNodeDiscoveryProvider; +import org.apache.druid.discovery.NodeRole; +import org.apache.druid.error.DruidException; +import org.apache.druid.guice.ManageLifecycle; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.FileUtils; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.lifecycle.LifecycleStart; +import org.apache.druid.java.util.common.lifecycle.LifecycleStop; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.msq.dart.controller.ControllerServerId; +import org.apache.druid.msq.dart.worker.http.DartWorkerInfo; +import org.apache.druid.msq.dart.worker.http.GetWorkersResponse; +import org.apache.druid.msq.exec.Worker; +import org.apache.druid.msq.indexing.error.CanceledFault; +import org.apache.druid.msq.indexing.error.MSQException; +import org.apache.druid.msq.rpc.ResourcePermissionMapper; +import org.apache.druid.msq.rpc.WorkerResource; +import org.apache.druid.query.QueryContext; +import org.apache.druid.server.security.AuthorizerMapper; +import org.joda.time.DateTime; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ExecutorService; +import java.util.stream.Collectors; + +@ManageLifecycle +public class DartWorkerRunner +{ + private static final Logger log = new Logger(DartWorkerRunner.class); + + /** + * Set of active controllers. Ignore requests from others. + */ + @GuardedBy("this") + private final Set activeControllerHosts = new HashSet<>(); + + /** + * Query ID -> Worker instance. + */ + @GuardedBy("this") + private final Map workerMap = new HashMap<>(); + private final DartWorkerFactory workerFactory; + private final ExecutorService workerExec; + private final DruidNodeDiscoveryProvider discoveryProvider; + private final ResourcePermissionMapper permissionMapper; + private final AuthorizerMapper authorizerMapper; + private final File baseTempDir; + + public DartWorkerRunner( + final DartWorkerFactory workerFactory, + final ExecutorService workerExec, + final DruidNodeDiscoveryProvider discoveryProvider, + final ResourcePermissionMapper permissionMapper, + final AuthorizerMapper authorizerMapper, + final File baseTempDir + ) + { + this.workerFactory = workerFactory; + this.workerExec = workerExec; + this.discoveryProvider = discoveryProvider; + this.permissionMapper = permissionMapper; + this.authorizerMapper = authorizerMapper; + this.baseTempDir = baseTempDir; + } + + /** + * Start a worker, creating a holder for it. If a worker with this query ID is already started, does nothing. + * Returns the worker. + * + * @throws DruidException if the controllerId does not correspond to a currently-active controller + */ + public Worker startWorker( + final String queryId, + final ControllerServerId controllerServerId, + final QueryContext context + ) + { + final WorkerHolder holder; + final boolean newHolder; + + synchronized (this) { + if (!activeControllerHosts.contains(controllerServerId.getHost())) { + throw DruidException.forPersona(DruidException.Persona.OPERATOR) + .ofCategory(DruidException.Category.RUNTIME_FAILURE) + .build("Received startWorker requested for unknown controller[%s]", controllerServerId); + } + + final WorkerHolder existingHolder = workerMap.get(queryId); + if (existingHolder != null) { + holder = existingHolder; + newHolder = false; + } else { + final Worker worker = workerFactory.build(queryId, controllerServerId.getHost(), baseTempDir, context); + final WorkerResource resource = new WorkerResource(worker, permissionMapper, authorizerMapper); + holder = new WorkerHolder(worker, controllerServerId, resource, DateTimes.nowUtc()); + workerMap.put(queryId, holder); + newHolder = true; + } + } + + if (newHolder) { + workerExec.submit(() -> { + final String originalThreadName = Thread.currentThread().getName(); + try { + Thread.currentThread().setName(StringUtils.format("%s[%s]", originalThreadName, queryId)); + holder.worker.run(); + } + catch (Throwable t) { + if (Thread.interrupted() + || t instanceof MSQException && ((MSQException) t).getFault().getErrorCode().equals(CanceledFault.CODE)) { + log.debug(t, "Canceled, exiting thread."); + } else { + log.warn(t, "Worker for query[%s] failed and stopped.", queryId); + } + } + finally { + synchronized (this) { + workerMap.remove(queryId, holder); + } + + Thread.currentThread().setName(originalThreadName); + } + }); + } + + return holder.worker; + } + + /** + * Synchronizes with controller state. + */ + public void stopWorker(final String queryId) + { + final WorkerHolder holder; + + synchronized (this) { + holder = workerMap.get(queryId); + } + + if (holder != null) { + holder.worker.stop(); + } + } + + /** + * Get the worker resource handler for a query ID if it exists. Returns null if the worker is not running. + */ + @Nullable + public WorkerResource getWorkerResource(final String queryId) + { + synchronized (this) { + final WorkerHolder holder = workerMap.get(queryId); + if (holder != null) { + return holder.resource; + } else { + return null; + } + } + } + + /** + * Returns a {@link GetWorkersResponse} with information about all active workers. + */ + public GetWorkersResponse getWorkersResponse() + { + final List infos = new ArrayList<>(); + + synchronized (this) { + for (final Map.Entry entry : workerMap.entrySet()) { + final String queryId = entry.getKey(); + final WorkerHolder workerHolder = entry.getValue(); + infos.add( + new DartWorkerInfo( + queryId, + WorkerId.fromString(workerHolder.worker.id()), + workerHolder.controllerServerId, + workerHolder.acceptTime + ) + ); + } + } + + return new GetWorkersResponse(infos); + } + + @LifecycleStart + public void start() + { + createAndCleanTempDirectory(); + + final DruidNodeDiscovery brokers = discoveryProvider.getForNodeRole(NodeRole.BROKER); + brokers.registerListener(new BrokerListener()); + } + + @LifecycleStop + public void stop() + { + synchronized (this) { + final Collection holders = workerMap.values(); + + for (final WorkerHolder holder : holders) { + holder.worker.stop(); + } + + for (final WorkerHolder holder : holders) { + holder.worker.awaitStop(); + } + } + } + + /** + * Creates the {@link #baseTempDir}, and removes any items in it that still exist. + */ + private void createAndCleanTempDirectory() + { + try { + FileUtils.mkdirp(baseTempDir); + } + catch (IOException e) { + throw new RuntimeException(e); + } + + final File[] files = baseTempDir.listFiles(); + + if (files != null) { + for (final File file : files) { + if (file.isDirectory()) { + try { + FileUtils.deleteDirectory(file); + log.info("Removed stale query directory[%s].", file); + } + catch (Exception e) { + log.noStackTrace().warn(e, "Could not remove stale query directory[%s], skipping.", file); + } + } + } + } + } + + private static class WorkerHolder + { + private final Worker worker; + private final WorkerResource resource; + private final ControllerServerId controllerServerId; + private final DateTime acceptTime; + + public WorkerHolder( + Worker worker, + ControllerServerId controllerServerId, + WorkerResource resource, + final DateTime acceptTime + ) + { + this.worker = worker; + this.resource = resource; + this.controllerServerId = controllerServerId; + this.acceptTime = acceptTime; + } + } + + /** + * Listener that cancels work associated with Brokers that have gone away. + */ + private class BrokerListener implements DruidNodeDiscovery.Listener + { + @Override + public void nodesAdded(Collection nodes) + { + synchronized (DartWorkerRunner.this) { + for (final DiscoveryDruidNode node : nodes) { + activeControllerHosts.add(node.getDruidNode().getHostAndPortToUse()); + } + } + } + + @Override + public void nodesRemoved(Collection nodes) + { + final Set hostsRemoved = + nodes.stream().map(node -> node.getDruidNode().getHostAndPortToUse()).collect(Collectors.toSet()); + + final List workersToNotify = new ArrayList<>(); + + synchronized (DartWorkerRunner.this) { + activeControllerHosts.removeAll(hostsRemoved); + + for (Map.Entry entry : workerMap.entrySet()) { + if (hostsRemoved.contains(entry.getValue().controllerServerId.getHost())) { + workersToNotify.add(entry.getValue().worker); + } + } + } + + for (final Worker worker : workersToNotify) { + worker.controllerFailed(); + } + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/QueryableDataSegment.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/QueryableDataSegment.java new file mode 100644 index 000000000000..4772e66808aa --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/QueryableDataSegment.java @@ -0,0 +1,86 @@ +/* + * 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.dart.worker; + +import com.google.common.base.Preconditions; +import org.apache.druid.timeline.DataSegment; +import org.joda.time.Interval; + +import java.util.Objects; + +public class QueryableDataSegment +{ + private final DataSegment segment; + private final Interval interval; + private final int workerNumber; + + public QueryableDataSegment(final DataSegment segment, final Interval interval, final int workerNumber) + { + this.segment = Preconditions.checkNotNull(segment, "segment"); + this.interval = Preconditions.checkNotNull(interval, "interval"); + this.workerNumber = workerNumber; + } + + public DataSegment getSegment() + { + return segment; + } + + public Interval getInterval() + { + return interval; + } + + public int getWorkerNumber() + { + return workerNumber; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + QueryableDataSegment that = (QueryableDataSegment) o; + return workerNumber == that.workerNumber + && Objects.equals(segment, that.segment) + && Objects.equals(interval, that.interval); + } + + @Override + public int hashCode() + { + return Objects.hash(segment, interval, workerNumber); + } + + @Override + public String toString() + { + return "QueryableDataSegment{" + + "segment=" + segment + + ", interval=" + interval + + ", workerNumber=" + workerNumber + + '}'; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/WorkerId.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/WorkerId.java new file mode 100644 index 000000000000..0f213dda33f0 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/WorkerId.java @@ -0,0 +1,157 @@ +/* + * 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.dart.worker; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonValue; +import com.google.common.base.Joiner; +import com.google.common.base.Preconditions; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.msq.dart.worker.http.DartWorkerResource; +import org.apache.druid.msq.kernel.controller.ControllerQueryKernelConfig; +import org.apache.druid.server.DruidNode; +import org.apache.druid.server.coordination.DruidServerMetadata; + +import java.net.URI; +import java.net.URISyntaxException; +import java.util.Objects; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * Worker IDs, of the type returned by {@link ControllerQueryKernelConfig#getWorkerIds()}. + * + * Dart workerIds are strings of the form "scheme:host:port:queryId", like + * "https:host1.example.com:8083:2f05528c-a882-4da5-8b7d-2ecafb7f3f4e". + */ +public class WorkerId +{ + private static final Pattern PATTERN = Pattern.compile("^(\\w+):(.+:\\d+):([a-z0-9-]+)$"); + + private final String scheme; + private final String hostAndPort; + private final String queryId; + private final String fullString; + + private WorkerId(final String scheme, final String hostAndPort, final String queryId) + { + this.scheme = Preconditions.checkNotNull(scheme, "scheme"); + this.hostAndPort = Preconditions.checkNotNull(hostAndPort, "hostAndPort"); + this.queryId = Preconditions.checkNotNull(queryId, "queryId"); + this.fullString = Joiner.on(':').join(scheme, hostAndPort, queryId); + } + + @JsonCreator + public static WorkerId fromString(final String s) + { + if (s == null) { + throw new IAE("Missing workerId"); + } + + final Matcher matcher = PATTERN.matcher(s); + if (matcher.matches()) { + return new WorkerId(matcher.group(1), matcher.group(2), matcher.group(3)); + } else { + throw new IAE("Invalid workerId[%s]", s); + } + } + + /** + * Create a worker ID, which is a URL. + */ + public static WorkerId fromDruidNode(final DruidNode node, final String queryId) + { + return new WorkerId( + node.getServiceScheme(), + node.getHostAndPort(), + queryId + ); + } + + /** + * Create a worker ID, which is a URL. + */ + public static WorkerId fromDruidServerMetadata(final DruidServerMetadata server, final String queryId) + { + return new WorkerId( + server.getHostAndTlsPort() != null ? "https" : "http", + server.getHost(), + queryId + ); + } + + public String getScheme() + { + return scheme; + } + + public String getHostAndPort() + { + return hostAndPort; + } + + public String getQueryId() + { + return queryId; + } + + public URI toUri() + { + try { + final String path = StringUtils.format( + "%s/workers/%s", + DartWorkerResource.PATH, + StringUtils.urlEncode(queryId) + ); + + return new URI(scheme, hostAndPort, path, null, null); + } + catch (URISyntaxException e) { + throw new RuntimeException(e); + } + } + + @Override + @JsonValue + public String toString() + { + return fullString; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + WorkerId workerId = (WorkerId) o; + return Objects.equals(fullString, workerId.fullString); + } + + @Override + public int hashCode() + { + return fullString.hashCode(); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/http/DartWorkerInfo.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/http/DartWorkerInfo.java new file mode 100644 index 000000000000..0af51846a5de --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/http/DartWorkerInfo.java @@ -0,0 +1,87 @@ +/* + * 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.dart.worker.http; + +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.msq.dart.controller.ControllerServerId; +import org.apache.druid.msq.dart.controller.http.DartQueryInfo; +import org.apache.druid.msq.dart.worker.WorkerId; +import org.joda.time.DateTime; + +/** + * Class included in {@link GetWorkersResponse}. + */ +public class DartWorkerInfo +{ + private final String dartQueryId; + private final WorkerId workerId; + private final ControllerServerId controllerServerId; + private final DateTime startTime; + + public DartWorkerInfo( + @JsonProperty("dartQueryId") final String dartQueryId, + @JsonProperty("workerId") final WorkerId workerId, + @JsonProperty("controllerServerId") final ControllerServerId controllerServerId, + @JsonProperty("startTime") final DateTime startTime + ) + { + this.dartQueryId = dartQueryId; + this.workerId = workerId; + this.controllerServerId = controllerServerId; + this.startTime = startTime; + } + + /** + * Dart query ID generated by the system. Globally unique. + */ + @JsonProperty + public String getDartQueryId() + { + return dartQueryId; + } + + /** + * Worker ID for this query. + */ + @JsonProperty + public WorkerId getWorkerId() + { + return workerId; + } + + /** + * Controller server that manages this query. + */ + @JsonProperty + public ControllerServerId getControllerHost() + { + return controllerServerId; + } + + /** + * Time this query was accepted by this worker. May be somewhat later than the {@link DartQueryInfo#getStartTime()} + * on the controller. + */ + @JsonProperty + public DateTime getStartTime() + { + return startTime; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/http/DartWorkerResource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/http/DartWorkerResource.java new file mode 100644 index 000000000000..0bd25143cf72 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/http/DartWorkerResource.java @@ -0,0 +1,184 @@ +/* + * 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.dart.worker.http; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; +import com.google.inject.Inject; +import org.apache.druid.error.DruidException; +import org.apache.druid.guice.LazySingleton; +import org.apache.druid.guice.annotations.Smile; +import org.apache.druid.messages.server.MessageRelayResource; +import org.apache.druid.messages.server.Outbox; +import org.apache.druid.msq.dart.Dart; +import org.apache.druid.msq.dart.controller.ControllerServerId; +import org.apache.druid.msq.dart.controller.messages.ControllerMessage; +import org.apache.druid.msq.dart.worker.DartWorkerClient; +import org.apache.druid.msq.dart.worker.DartWorkerRunner; +import org.apache.druid.msq.exec.Worker; +import org.apache.druid.msq.kernel.WorkOrder; +import org.apache.druid.msq.rpc.MSQResourceUtils; +import org.apache.druid.msq.rpc.ResourcePermissionMapper; +import org.apache.druid.msq.rpc.WorkerResource; +import org.apache.druid.server.security.AuthorizerMapper; + +import javax.servlet.http.HttpServletRequest; +import javax.ws.rs.Consumes; +import javax.ws.rs.GET; +import javax.ws.rs.POST; +import javax.ws.rs.Path; +import javax.ws.rs.PathParam; +import javax.ws.rs.Produces; +import javax.ws.rs.core.Context; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; + +/** + * Subclass of {@link WorkerResource} suitable for usage on a Historical. + * + * Note that this is not the same resource as used by {@link org.apache.druid.msq.indexing.MSQWorkerTask}. + * For that, see {@link org.apache.druid.msq.indexing.client.WorkerChatHandler}. + */ +@LazySingleton +@Path(DartWorkerResource.PATH + '/') +public class DartWorkerResource +{ + /** + * Root of worker APIs. + */ + public static final String PATH = "/druid/v2/dart-worker"; + + /** + * Header containing the controller ID, from {@link ControllerServerId}. + */ + public static final String HEADER_CONTROLLER_ID = "X-Dart-Controller-Id"; + + private final DartWorkerRunner workerRunner; + private final ResourcePermissionMapper permissionMapper; + private final AuthorizerMapper authorizerMapper; + private final MessageRelayResource messageRelayResource; + + @Inject + public DartWorkerResource( + final DartWorkerRunner workerRunner, + @Dart final ResourcePermissionMapper permissionMapper, + @Smile final ObjectMapper smileMapper, + final Outbox outbox, + final AuthorizerMapper authorizerMapper + ) + { + this.workerRunner = workerRunner; + this.permissionMapper = permissionMapper; + this.authorizerMapper = authorizerMapper; + this.messageRelayResource = new MessageRelayResource<>( + outbox, + smileMapper, + ControllerMessage.class + ); + } + + /** + * API for retrieving all currently-running queries. + */ + @GET + @Produces(MediaType.APPLICATION_JSON) + @Path("/workers") + public GetWorkersResponse httpGetWorkers(@Context final HttpServletRequest req) + { + MSQResourceUtils.authorizeAdminRequest(permissionMapper, authorizerMapper, req); + return workerRunner.getWorkersResponse(); + } + + /** + * Like {@link WorkerResource#httpPostWorkOrder(WorkOrder, HttpServletRequest)}, but implicitly starts a worker + * when the work order is posted. Shadows {@link WorkerResource#httpPostWorkOrder(WorkOrder, HttpServletRequest)}. + */ + @POST + @Consumes({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) + @Path("/workers/{queryId}/workOrder") + public Response httpPostWorkOrder( + final WorkOrder workOrder, + @PathParam("queryId") final String queryId, + @Context final HttpServletRequest req + ) + { + MSQResourceUtils.authorizeAdminRequest(permissionMapper, authorizerMapper, req); + final String controllerIdString = req.getHeader(HEADER_CONTROLLER_ID); + if (controllerIdString == null) { + throw DruidException.forPersona(DruidException.Persona.DEVELOPER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build("Missing controllerId[%s]", HEADER_CONTROLLER_ID); + } + + workerRunner.startWorker(queryId, ControllerServerId.fromString(controllerIdString), workOrder.getWorkerContext()) + .postWorkOrder(workOrder); + + return Response.status(Response.Status.ACCEPTED).build(); + } + + /** + * Stops a worker. Returns immediately; does not wait for the worker to actually finish. + * + * It is very important that this, or {@link Worker#postFinish()}, is called on the conclusion of each query. + * For this reason, {@link DartWorkerClient} uses an unlimited retry policy. If a stop command was lost, a worker + * could run in a zombie state without its controller. This state would persist until the server that ran the + * controller shuts down or restarts. At that time, the listener in {@link DartWorkerRunner.BrokerListener} calls + * {@link Worker#controllerFailed()}, and the zombie worker would exit. + */ + @POST + @Path("/workers/{queryId}/stop") + public Response httpPostStopWorker( + @PathParam("queryId") final String queryId, + @Context final HttpServletRequest req + ) + { + MSQResourceUtils.authorizeAdminRequest(permissionMapper, authorizerMapper, req); + workerRunner.stopWorker(queryId); + return Response.status(Response.Status.ACCEPTED).build(); + } + + /** + * Handles all {@link WorkerResource} calls, except {@link WorkerResource#httpPostWorkOrder}, which is handled + * by {@link #httpPostWorkOrder(WorkOrder, String, HttpServletRequest)}. + */ + @Path("/workers/{queryId}") + public Object httpCallWorker( + @PathParam("queryId") final String queryId, + @Context final HttpServletRequest req + ) + { + final WorkerResource resource = workerRunner.getWorkerResource(queryId); + + if (resource != null) { + return resource; + } else { + // Return HTTP 503 (Service Unavailable) so clients retry. When workers are first starting up and contacting + // each other, worker A may contact worker B before worker B has started up. + return Response.status(Response.Status.SERVICE_UNAVAILABLE).build(); + } + } + + @Path("/relay") + public Object httpCallMessageServer(@Context final HttpServletRequest req) + { + MSQResourceUtils.authorizeAdminRequest(permissionMapper, authorizerMapper, req); + return messageRelayResource; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/http/GetWorkersResponse.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/http/GetWorkersResponse.java new file mode 100644 index 000000000000..f1618a08b54e --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/dart/worker/http/GetWorkersResponse.java @@ -0,0 +1,44 @@ +/* + * 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.dart.worker.http; + +import com.fasterxml.jackson.annotation.JsonProperty; + +import javax.servlet.http.HttpServletRequest; +import java.util.List; + +/** + * Response from {@link DartWorkerResource#httpGetWorkers(HttpServletRequest)}, the "get all workers" API. + */ +public class GetWorkersResponse +{ + private final List workers; + + public GetWorkersResponse(@JsonProperty("workers") final List workers) + { + this.workers = workers; + } + + @JsonProperty + public List getWorkers() + { + return workers; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Controller.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Controller.java index d316b9b6b0b7..3c770586d08b 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Controller.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Controller.java @@ -121,6 +121,11 @@ void resultsComplete( */ List getWorkerIds(); + /** + * Returns whether this controller has a worker with the given ID. + */ + boolean hasWorker(final String workerId); + @Nullable TaskReport.ReportMap liveReports(); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index 72d8216088fe..e164ff28bc93 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -1173,6 +1173,16 @@ public List getWorkerIds() return workerManager.getWorkerIds(); } + @Override + public boolean hasWorker(String workerId) + { + if (workerManager == null) { + return false; + } + + return workerManager.getWorkerNumber(workerId) != WorkerManager.UNKNOWN_WORKER_NUMBER; + } + @SuppressWarnings({"unchecked", "rawtypes"}) @Nullable private Int2ObjectMap makeWorkerFactoryInfosForStage( diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerManager.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerManager.java index ebce4821d591..31af0953d2f9 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerManager.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerManager.java @@ -83,8 +83,11 @@ public interface WorkerManager Map> getWorkerStats(); /** - * Blocks until all workers exit. Returns quietly, no matter whether there was an exception associated with the - * future from {@link #start()} or not. + * Stop all workers. + * + * The task-based implementation blocks until all tasks exit. Dart's implementation queues workers for stopping in + * the background, and returns immediately. Either way, this method returns quietly, no matter whether there was an + * exception associated with the future from {@link #start()} or not. * * @param interrupt whether to interrupt currently-running work */ diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/WorkerResource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/WorkerResource.java index a0bfecff5427..f4cd2f534e10 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/WorkerResource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/WorkerResource.java @@ -56,6 +56,7 @@ import javax.ws.rs.core.StreamingOutput; import java.io.InputStream; import java.io.OutputStream; +import java.util.concurrent.atomic.AtomicBoolean; public class WorkerResource { @@ -104,6 +105,8 @@ public Response httpGetChannelData( worker.readStageOutput(new StageId(queryId, stageNumber), partitionNumber, offset); final AsyncContext asyncContext = req.startAsync(); + final AtomicBoolean responseResolved = new AtomicBoolean(); + asyncContext.setTimeout(GET_CHANNEL_DATA_TIMEOUT); asyncContext.addListener( new AsyncListener() @@ -116,6 +119,10 @@ public void onComplete(AsyncEvent event) @Override public void onTimeout(AsyncEvent event) { + if (responseResolved.compareAndSet(false, true)) { + return; + } + HttpServletResponse response = (HttpServletResponse) asyncContext.getResponse(); response.setStatus(HttpServletResponse.SC_OK); event.getAsyncContext().complete(); @@ -144,7 +151,11 @@ public void onStartAsync(AsyncEvent event) @Override public void onSuccess(final InputStream inputStream) { - HttpServletResponse response = (HttpServletResponse) asyncContext.getResponse(); + if (!responseResolved.compareAndSet(false, true)) { + return; + } + + final HttpServletResponse response = (HttpServletResponse) asyncContext.getResponse(); try (final OutputStream outputStream = response.getOutputStream()) { if (inputStream == null) { @@ -188,7 +199,7 @@ public void onSuccess(final InputStream inputStream) @Override public void onFailure(Throwable e) { - if (!dataFuture.isCancelled()) { + if (responseResolved.compareAndSet(false, true)) { try { HttpServletResponse response = (HttpServletResponse) asyncContext.getResponse(); response.sendError(HttpServletResponse.SC_INTERNAL_SERVER_ERROR); 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 202c1c591b10..7cf8201c5252 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 @@ -28,6 +28,7 @@ import org.apache.druid.error.DruidException; import org.apache.druid.error.InvalidInput; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; @@ -56,7 +57,6 @@ import org.apache.druid.sql.calcite.parser.DruidSqlIngest; import org.apache.druid.sql.calcite.parser.DruidSqlInsert; import org.apache.druid.sql.calcite.parser.DruidSqlReplace; -import org.apache.druid.sql.calcite.planner.ColumnMapping; import org.apache.druid.sql.calcite.planner.ColumnMappings; import org.apache.druid.sql.calcite.planner.PlannerContext; import org.apache.druid.sql.calcite.planner.QueryUtils; @@ -96,7 +96,6 @@ public class MSQTaskQueryMaker implements QueryMaker private final List> fieldMapping; private final MSQTerminalStageSpecFactory terminalStageSpecFactory; - MSQTaskQueryMaker( @Nullable final IngestDestination targetDataSource, final OverlordClient overlordClient, @@ -122,6 +121,38 @@ public QueryResponse runQuery(final DruidQuery druidQuery) String taskId = MSQTasks.controllerTaskId(plannerContext.getSqlQueryId()); + final Map taskContext = new HashMap<>(); + taskContext.put(LookupLoadingSpec.CTX_LOOKUP_LOADING_MODE, plannerContext.getLookupLoadingSpec().getMode()); + if (plannerContext.getLookupLoadingSpec().getMode() == LookupLoadingSpec.Mode.ONLY_REQUIRED) { + taskContext.put(LookupLoadingSpec.CTX_LOOKUPS_TO_LOAD, plannerContext.getLookupLoadingSpec().getLookupsToLoad()); + } + + final List> typeList = getTypes(druidQuery, fieldMapping, plannerContext); + + final MSQControllerTask controllerTask = new MSQControllerTask( + taskId, + makeQuerySpec(targetDataSource, druidQuery, fieldMapping, plannerContext, terminalStageSpecFactory), + MSQTaskQueryMakerUtils.maskSensitiveJsonKeys(plannerContext.getSql()), + plannerContext.queryContextMap(), + SqlResults.Context.fromPlannerContext(plannerContext), + typeList.stream().map(typeInfo -> typeInfo.lhs).collect(Collectors.toList()), + typeList.stream().map(typeInfo -> typeInfo.rhs).collect(Collectors.toList()), + taskContext + ); + + FutureUtils.getUnchecked(overlordClient.runTask(taskId, controllerTask), true); + return QueryResponse.withEmptyContext(Sequences.simple(Collections.singletonList(new Object[]{taskId}))); + } + + public static MSQSpec makeQuerySpec( + @Nullable final IngestDestination targetDataSource, + final DruidQuery druidQuery, + final List> fieldMapping, + final PlannerContext plannerContext, + final MSQTerminalStageSpecFactory terminalStageSpecFactory + ) + { + // SQL query context: context provided by the user, and potentially modified by handlers during planning. // Does not directly influence task execution, but it does form the basis for the initial native query context, // which *does* influence task execution. @@ -138,23 +169,18 @@ public QueryResponse runQuery(final DruidQuery druidQuery) MSQMode.populateDefaultQueryContext(msqMode, nativeQueryContext); } - Object segmentGranularity; - try { - segmentGranularity = Optional.ofNullable(plannerContext.queryContext() - .get(DruidSqlInsert.SQL_INSERT_SEGMENT_GRANULARITY)) - .orElse(jsonMapper.writeValueAsString(DEFAULT_SEGMENT_GRANULARITY)); - } - catch (JsonProcessingException e) { - // This would only be thrown if we are unable to serialize the DEFAULT_SEGMENT_GRANULARITY, which we don't expect - // to happen - throw DruidException.defensive() - .build( - e, - "Unable to deserialize the DEFAULT_SEGMENT_GRANULARITY in MSQTaskQueryMaker. " - + "This shouldn't have happened since the DEFAULT_SEGMENT_GRANULARITY object is guaranteed to be " - + "serializable. Please raise an issue in case you are seeing this message while executing a query." - ); - } + Object segmentGranularity = + Optional.ofNullable(plannerContext.queryContext().get(DruidSqlInsert.SQL_INSERT_SEGMENT_GRANULARITY)) + .orElseGet(() -> { + try { + return plannerContext.getJsonMapper().writeValueAsString(DEFAULT_SEGMENT_GRANULARITY); + } + catch (JsonProcessingException e) { + // This would only be thrown if we are unable to serialize the DEFAULT_SEGMENT_GRANULARITY, + // which we don't expect to happen. + throw DruidException.defensive().build(e, "Unable to serialize DEFAULT_SEGMENT_GRANULARITY"); + } + }); final int maxNumTasks = MultiStageQueryContext.getMaxNumTasks(sqlQueryContext); @@ -170,7 +196,7 @@ public QueryResponse runQuery(final DruidQuery druidQuery) final int rowsPerSegment = MultiStageQueryContext.getRowsPerSegment(sqlQueryContext); final int maxRowsInMemory = MultiStageQueryContext.getRowsInMemory(sqlQueryContext); final Integer maxNumSegments = MultiStageQueryContext.getMaxNumSegments(sqlQueryContext); - final IndexSpec indexSpec = MultiStageQueryContext.getIndexSpec(sqlQueryContext, jsonMapper); + final IndexSpec indexSpec = MultiStageQueryContext.getIndexSpec(sqlQueryContext, plannerContext.getJsonMapper()); final boolean finalizeAggregations = MultiStageQueryContext.isFinalizeAggregations(sqlQueryContext); final List replaceTimeChunks = @@ -193,29 +219,6 @@ public QueryResponse runQuery(final DruidQuery druidQuery) ) .orElse(null); - // For assistance computing return types if !finalizeAggregations. - final Map aggregationIntermediateTypeMap = - 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 Entry entry : fieldMapping) { - final String queryColumn = druidQuery.getOutputRowSignature().getColumnName(entry.getKey()); - - final SqlTypeName sqlTypeName; - - if (!finalizeAggregations && aggregationIntermediateTypeMap.containsKey(queryColumn)) { - final ColumnType druidType = aggregationIntermediateTypeMap.get(queryColumn); - sqlTypeName = new RowSignatures.ComplexSqlType(SqlTypeName.OTHER, druidType, true).getSqlTypeName(); - } 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; if (targetDataSource instanceof ExportDestination) { @@ -229,7 +232,8 @@ public QueryResponse runQuery(final DruidQuery druidQuery) } else if (targetDataSource instanceof TableDestination) { Granularity segmentGranularityObject; try { - segmentGranularityObject = jsonMapper.readValue((String) segmentGranularity, Granularity.class); + segmentGranularityObject = + plannerContext.getJsonMapper().readValue((String) segmentGranularity, Granularity.class); } catch (Exception e) { throw DruidException.defensive() @@ -288,7 +292,7 @@ public QueryResponse runQuery(final DruidQuery druidQuery) final MSQSpec querySpec = MSQSpec.builder() .query(druidQuery.getQuery().withOverriddenContext(nativeQueryContextOverrides)) - .columnMappings(new ColumnMappings(columnMappings)) + .columnMappings(new ColumnMappings(QueryUtils.buildColumnMappings(fieldMapping, druidQuery))) .destination(destination) .assignmentStrategy(MultiStageQueryContext.getAssignmentStrategy(sqlQueryContext)) .tuningConfig(new MSQTuningConfig(maxNumWorkers, maxRowsInMemory, rowsPerSegment, maxNumSegments, indexSpec)) @@ -296,25 +300,42 @@ public QueryResponse runQuery(final DruidQuery druidQuery) MSQTaskQueryMakerUtils.validateRealtimeReindex(querySpec); - final Map context = new HashMap<>(); - context.put(LookupLoadingSpec.CTX_LOOKUP_LOADING_MODE, plannerContext.getLookupLoadingSpec().getMode()); - if (plannerContext.getLookupLoadingSpec().getMode() == LookupLoadingSpec.Mode.ONLY_REQUIRED) { - context.put(LookupLoadingSpec.CTX_LOOKUPS_TO_LOAD, plannerContext.getLookupLoadingSpec().getLookupsToLoad()); - } + return querySpec.withOverriddenContext(nativeQueryContext); + } - final MSQControllerTask controllerTask = new MSQControllerTask( - taskId, - querySpec.withOverriddenContext(nativeQueryContext), - MSQTaskQueryMakerUtils.maskSensitiveJsonKeys(plannerContext.getSql()), - plannerContext.queryContextMap(), - SqlResults.Context.fromPlannerContext(plannerContext), - sqlTypeNames, - columnTypeList, - context - ); + public static List> getTypes( + final DruidQuery druidQuery, + final List> fieldMapping, + final PlannerContext plannerContext + ) + { + final boolean finalizeAggregations = MultiStageQueryContext.isFinalizeAggregations(plannerContext.queryContext()); - FutureUtils.getUnchecked(overlordClient.runTask(taskId, controllerTask), true); - return QueryResponse.withEmptyContext(Sequences.simple(Collections.singletonList(new Object[]{taskId}))); + // For assistance computing return types if !finalizeAggregations. + final Map aggregationIntermediateTypeMap = + finalizeAggregations ? null /* Not needed */ : buildAggregationIntermediateTypeMap(druidQuery); + + final List> retVal = new ArrayList<>(); + + for (final Entry entry : fieldMapping) { + final String queryColumn = druidQuery.getOutputRowSignature().getColumnName(entry.getKey()); + + final SqlTypeName sqlTypeName; + + if (!finalizeAggregations && aggregationIntermediateTypeMap.containsKey(queryColumn)) { + final ColumnType druidType = aggregationIntermediateTypeMap.get(queryColumn); + sqlTypeName = new RowSignatures.ComplexSqlType(SqlTypeName.OTHER, druidType, true).getSqlTypeName(); + } else { + sqlTypeName = druidQuery.getOutputRowType().getFieldList().get(entry.getKey()).getType().getSqlTypeName(); + } + + final ColumnType columnType = + druidQuery.getOutputRowSignature().getColumnType(queryColumn).orElse(ColumnType.STRING); + + retVal.add(Pair.of(sqlTypeName, columnType)); + } + + return retVal; } private static Map buildAggregationIntermediateTypeMap(final DruidQuery druidQuery) 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 bdebe32a16fb..f8d73322190e 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 @@ -113,13 +113,21 @@ public void validateContext(Map queryContext) } @Override - public RelDataType resultTypeForSelect(RelDataTypeFactory typeFactory, RelDataType validatedRowType) + public RelDataType resultTypeForSelect( + RelDataTypeFactory typeFactory, + RelDataType validatedRowType, + Map queryContext + ) { return getMSQStructType(typeFactory); } @Override - public RelDataType resultTypeForInsert(RelDataTypeFactory typeFactory, RelDataType validatedRowType) + public RelDataType resultTypeForInsert( + RelDataTypeFactory typeFactory, + RelDataType validatedRowType, + Map queryContext + ) { return getMSQStructType(typeFactory); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MSQTaskQueryMakerUtils.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MSQTaskQueryMakerUtils.java index a30c9bb0aec0..36c90a21f002 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MSQTaskQueryMakerUtils.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MSQTaskQueryMakerUtils.java @@ -28,7 +28,6 @@ import org.apache.druid.msq.indexing.MSQSpec; import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination; -import java.util.HashSet; import java.util.List; import java.util.Set; import java.util.regex.Matcher; @@ -82,10 +81,8 @@ public static void validateContextSortOrderColumnsExist( final Set allOutputColumns ) { - final Set allOutputColumnsSet = new HashSet<>(allOutputColumns); - for (final String column : contextSortOrder) { - if (!allOutputColumnsSet.contains(column)) { + if (!allOutputColumns.contains(column)) { throw InvalidSqlInput.exception( "Column[%s] from context parameter[%s] does not appear in the query output", column, diff --git a/extensions-core/multi-stage-query/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule b/extensions-core/multi-stage-query/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule index 92be5604cb8a..1058d5d5f99e 100644 --- a/extensions-core/multi-stage-query/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule +++ b/extensions-core/multi-stage-query/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule @@ -13,6 +13,10 @@ # See the License for the specific language governing permissions and # limitations under the License. +org.apache.druid.msq.dart.guice.DartControllerMemoryManagementModule +org.apache.druid.msq.dart.guice.DartControllerModule +org.apache.druid.msq.dart.guice.DartWorkerMemoryManagementModule +org.apache.druid.msq.dart.guice.DartWorkerModule org.apache.druid.msq.guice.IndexerMemoryManagementModule org.apache.druid.msq.guice.MSQDurableStorageModule org.apache.druid.msq.guice.MSQExternalDataSourceModule diff --git a/processing/src/main/java/org/apache/druid/io/LimitedOutputStream.java b/processing/src/main/java/org/apache/druid/io/LimitedOutputStream.java new file mode 100644 index 000000000000..dd782d32097e --- /dev/null +++ b/processing/src/main/java/org/apache/druid/io/LimitedOutputStream.java @@ -0,0 +1,74 @@ +/* + * 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.io; + +import org.apache.druid.java.util.common.IOE; + +import java.io.FilterOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.util.function.Function; + +/** + * An {@link OutputStream} that limits how many bytes can be written. Throws {@link IOException} if the limit + * is exceeded. + */ +public class LimitedOutputStream extends FilterOutputStream +{ + private final long limit; + private final Function exceptionMessageFn; + long written; + + public LimitedOutputStream(OutputStream out, long limit, Function exceptionMessageFn) + { + super(out); + this.limit = limit; + this.exceptionMessageFn = exceptionMessageFn; + } + + @Override + public void write(int b) throws IOException + { + plus(1); + super.write(b); + } + + @Override + public void write(byte[] b) throws IOException + { + plus(b.length); + super.write(b); + } + + @Override + public void write(byte[] b, int off, int len) throws IOException + { + plus(len); + super.write(b, off, len); + } + + private void plus(final int n) throws IOException + { + written += n; + if (written > limit) { + throw new IOE(exceptionMessageFn.apply(limit)); + } + } +} diff --git a/server/src/main/java/org/apache/druid/client/BrokerServerView.java b/server/src/main/java/org/apache/druid/client/BrokerServerView.java index 2cb2bec03b59..546340f5c2ca 100644 --- a/server/src/main/java/org/apache/druid/client/BrokerServerView.java +++ b/server/src/main/java/org/apache/druid/client/BrokerServerView.java @@ -44,6 +44,7 @@ import org.apache.druid.timeline.VersionedIntervalTimeline; import org.apache.druid.timeline.partition.PartitionChunk; +import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -398,6 +399,19 @@ private void runTimelineCallbacks(final Function getDruidServerMetadata() + { + // Override default implementation for better performance. + final List retVal = new ArrayList<>(clients.size()); + + for (final QueryableDruidServer server : clients.values()) { + retVal.add(server.getServer().getMetadata()); + } + + return retVal; + } + @Override public List getDruidServers() { diff --git a/server/src/main/java/org/apache/druid/client/TimelineServerView.java b/server/src/main/java/org/apache/druid/client/TimelineServerView.java index 9a2b7b767755..7fa12e0f0f81 100644 --- a/server/src/main/java/org/apache/druid/client/TimelineServerView.java +++ b/server/src/main/java/org/apache/druid/client/TimelineServerView.java @@ -27,6 +27,7 @@ import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.TimelineLookup; +import java.util.ArrayList; import java.util.List; import java.util.Optional; import java.util.concurrent.Executor; @@ -48,7 +49,20 @@ public interface TimelineServerView extends ServerView Optional> getTimeline(DataSourceAnalysis analysis); /** - * Returns a list of {@link ImmutableDruidServer} + * Returns a snapshot of the current set of server metadata. + */ + default List getDruidServerMetadata() + { + final List druidServers = getDruidServers(); + final List metadatas = new ArrayList<>(druidServers.size()); + for (final ImmutableDruidServer druidServer : druidServers) { + metadatas.add(druidServer.getMetadata()); + } + return metadatas; + } + + /** + * Returns a snapshot of the current servers, their metadata, and their inventory. */ List getDruidServers(); diff --git a/server/src/main/java/org/apache/druid/messages/MessageBatch.java b/server/src/main/java/org/apache/druid/messages/MessageBatch.java new file mode 100644 index 000000000000..51209ff6d243 --- /dev/null +++ b/server/src/main/java/org/apache/druid/messages/MessageBatch.java @@ -0,0 +1,112 @@ +/* + * 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.messages; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.messages.client.MessageRelay; +import org.apache.druid.messages.server.MessageRelayResource; +import org.apache.druid.messages.server.Outbox; + +import java.util.List; +import java.util.Objects; + +/** + * A batch of messages collected by {@link MessageRelay} from a remote {@link Outbox} through + * {@link MessageRelayResource#httpGetMessagesFromOutbox}. + */ +public class MessageBatch +{ + private final List messages; + private final long epoch; + private final long startWatermark; + + @JsonCreator + public MessageBatch( + @JsonProperty("messages") final List messages, + @JsonProperty("epoch") final long epoch, + @JsonProperty("watermark") final long startWatermark + ) + { + this.messages = messages; + this.epoch = epoch; + this.startWatermark = startWatermark; + } + + /** + * The messages. + */ + @JsonProperty + public List getMessages() + { + return messages; + } + + /** + * Epoch, which is associated with a specific instance of {@link Outbox}. + */ + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_DEFAULT) + public long getEpoch() + { + return epoch; + } + + /** + * Watermark, an incrementing message ID that enables clients and servers to stay in sync, and enables + * acknowledging of messages. + */ + @JsonProperty("watermark") + @JsonInclude(JsonInclude.Include.NON_DEFAULT) + public long getStartWatermark() + { + return startWatermark; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + MessageBatch that = (MessageBatch) o; + return epoch == that.epoch && startWatermark == that.startWatermark && Objects.equals(messages, that.messages); + } + + @Override + public int hashCode() + { + return Objects.hash(messages, epoch, startWatermark); + } + + @Override + public String toString() + { + return "MessageBatch{" + + "messages=" + messages + + ", epoch=" + epoch + + ", startWatermark=" + startWatermark + + '}'; + } +} diff --git a/server/src/main/java/org/apache/druid/messages/client/MessageListener.java b/server/src/main/java/org/apache/druid/messages/client/MessageListener.java new file mode 100644 index 000000000000..6711c418f812 --- /dev/null +++ b/server/src/main/java/org/apache/druid/messages/client/MessageListener.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.messages.client; + +import org.apache.druid.server.DruidNode; + +/** + * Listener for messages received by clients. + */ +public interface MessageListener +{ + /** + * Called when a server is added. + * + * @param node server node + */ + void serverAdded(DruidNode node); + + /** + * Called when a message is received. Should not throw exceptions. If this method does throw an exception, + * the exception is logged and the message is acknowledged anyway. + * + * @param message the message that was received + */ + void messageReceived(MessageType message); + + /** + * Called when a server is removed. + * + * @param node server node + */ + void serverRemoved(DruidNode node); +} diff --git a/server/src/main/java/org/apache/druid/messages/client/MessageRelay.java b/server/src/main/java/org/apache/druid/messages/client/MessageRelay.java new file mode 100644 index 000000000000..deda87c7d23d --- /dev/null +++ b/server/src/main/java/org/apache/druid/messages/client/MessageRelay.java @@ -0,0 +1,243 @@ +/* + * 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.messages.client; + +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.messages.MessageBatch; +import org.apache.druid.messages.server.MessageRelayResource; +import org.apache.druid.rpc.ServiceClosedException; +import org.apache.druid.server.DruidNode; + +import java.io.Closeable; +import java.util.concurrent.CancellationException; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; + +/** + * Relays run on clients, and receive messages from a server. + * Uses {@link MessageRelayClient} to communicate with the {@link MessageRelayResource} on a server. + * that flows upstream + */ +public class MessageRelay implements Closeable +{ + private static final Logger log = new Logger(MessageRelay.class); + + /** + * Value to provide for epoch on the initial call to {@link MessageRelayClient#getMessages(String, long, long)}. + */ + public static final long INIT = -1; + + private final String selfHost; + private final DruidNode serverNode; + private final MessageRelayClient client; + private final AtomicBoolean closed = new AtomicBoolean(false); + private final Collector collector; + + public MessageRelay( + final String selfHost, + final DruidNode serverNode, + final MessageRelayClient client, + final MessageListener listener + ) + { + this.selfHost = selfHost; + this.serverNode = serverNode; + this.client = client; + this.collector = new Collector(listener); + } + + /** + * Start the {@link Collector}. + */ + public void start() + { + collector.start(); + } + + /** + * Stop the {@link Collector}. + */ + @Override + public void close() + { + if (closed.compareAndSet(false, true)) { + collector.stop(); + } + } + + /** + * Retrieves messages that are being sent to this client and hands them to {@link #listener}. + */ + private class Collector + { + private final MessageListener listener; + private final AtomicLong epoch = new AtomicLong(INIT); + private final AtomicLong watermark = new AtomicLong(INIT); + private final AtomicReference> currentCall = new AtomicReference<>(); + + public Collector(final MessageListener listener) + { + this.listener = listener; + } + + private void start() + { + if (!watermark.compareAndSet(INIT, 0)) { + throw new ISE("Already started"); + } + + listener.serverAdded(serverNode); + issueNextGetMessagesCall(); + } + + private void issueNextGetMessagesCall() + { + if (closed.get()) { + return; + } + + final long theEpoch = epoch.get(); + final long theWatermark = watermark.get(); + + log.debug( + "Getting messages from server[%s] for client[%s] (current state: epoch[%s] watermark[%s]).", + serverNode.getHostAndPortToUse(), + selfHost, + theEpoch, + theWatermark + ); + + final ListenableFuture> future = client.getMessages(selfHost, theEpoch, theWatermark); + + if (!currentCall.compareAndSet(null, future)) { + log.error( + "Fatal error: too many outgoing calls to server[%s] for client[%s] " + + "(current state: epoch[%s] watermark[%s]). Closing collector.", + serverNode.getHostAndPortToUse(), + selfHost, + theEpoch, + theWatermark + ); + + close(); + return; + } + + Futures.addCallback( + future, + new FutureCallback>() + { + @Override + public void onSuccess(final MessageBatch result) + { + log.debug("Received message batch: %s", result); + currentCall.compareAndSet(future, null); + final long endWatermark = result.getStartWatermark() + result.getMessages().size(); + if (theEpoch == INIT) { + epoch.set(result.getEpoch()); + watermark.set(endWatermark); + } else if (epoch.get() != result.getEpoch() + || !watermark.compareAndSet(result.getStartWatermark(), endWatermark)) { + // We don't expect to see this unless there is somehow another collector running with the same + // clientHost. If the unexpected happens, log it and close the collector. It will stay, doing + // nothing, in the MessageCollectors map until it is removed by the discovery listener. + log.error( + "Incorrect epoch + watermark from server[%s] for client[%s] " + + "(expected[%s:%s] but got[%s:%s]). " + + "Closing collector.", + serverNode.getHostAndPortToUse(), + selfHost, + theEpoch, + theWatermark, + result.getEpoch(), + result.getStartWatermark() + ); + + close(); + return; + } + + for (final MessageType message : result.getMessages()) { + try { + listener.messageReceived(message); + } + catch (Throwable e) { + log.warn( + e, + "Failed to handle message[%s] from server[%s] for client[%s].", + message, + selfHost, + serverNode.getHostAndPortToUse() + ); + } + } + + issueNextGetMessagesCall(); + } + + @Override + public void onFailure(final Throwable e) + { + currentCall.compareAndSet(future, null); + if (!(e instanceof CancellationException) && !(e instanceof ServiceClosedException)) { + // We don't expect to see any other errors, since we use an unlimited retry policy for clients. If the + // unexpected happens, log it and close the collector. It will stay, doing nothing, in the + // MessageCollectors map until it is removed by the discovery listener. + log.error( + e, + "Fatal error contacting server[%s] for client[%s] " + + "(current state: epoch[%s] watermark[%s]). " + + "Closing collector.", + serverNode.getHostAndPortToUse(), + selfHost, + theEpoch, + theWatermark + ); + } + + close(); + } + }, + Execs.directExecutor() + ); + } + + public void stop() + { + final ListenableFuture future = currentCall.getAndSet(null); + if (future != null) { + future.cancel(true); + } + + try { + listener.serverRemoved(serverNode); + } + catch (Throwable e) { + log.warn(e, "Failed to close server[%s]", serverNode.getHostAndPortToUse()); + } + } + } +} diff --git a/server/src/main/java/org/apache/druid/messages/client/MessageRelayClient.java b/server/src/main/java/org/apache/druid/messages/client/MessageRelayClient.java new file mode 100644 index 000000000000..fad228f7b5f0 --- /dev/null +++ b/server/src/main/java/org/apache/druid/messages/client/MessageRelayClient.java @@ -0,0 +1,43 @@ +/* + * 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.messages.client; + +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.messages.MessageBatch; +import org.apache.druid.messages.server.MessageRelayResource; + +/** + * Client for {@link MessageRelayResource}. + */ +public interface MessageRelayClient +{ + /** + * Get the next batch of messages from an outbox. + * + * @param clientHost which outbox to retrieve messages from. Each clientHost has its own outbox. + * @param epoch outbox epoch, or {@link MessageRelay#INIT} if this is the first call from the collector. + * @param startWatermark outbox message watermark to retrieve from. + * + * @return future that resolves to the next batch of messages + * + * @see MessageRelayResource#httpGetMessagesFromOutbox http endpoint this method calls + */ + ListenableFuture> getMessages(String clientHost, long epoch, long startWatermark); +} diff --git a/server/src/main/java/org/apache/druid/messages/client/MessageRelayClientImpl.java b/server/src/main/java/org/apache/druid/messages/client/MessageRelayClientImpl.java new file mode 100644 index 000000000000..140bd45e1af4 --- /dev/null +++ b/server/src/main/java/org/apache/druid/messages/client/MessageRelayClientImpl.java @@ -0,0 +1,85 @@ +/* + * 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.messages.client; + +import com.fasterxml.jackson.databind.JavaType; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.jackson.JacksonUtils; +import org.apache.druid.java.util.http.client.response.BytesFullResponseHandler; +import org.apache.druid.messages.MessageBatch; +import org.apache.druid.rpc.RequestBuilder; +import org.apache.druid.rpc.ServiceClient; +import org.eclipse.jetty.http.HttpStatus; +import org.jboss.netty.handler.codec.http.HttpMethod; + +import java.util.Collections; + +/** + * Production implementation of {@link MessageRelayClient}. + */ +public class MessageRelayClientImpl implements MessageRelayClient +{ + private final ServiceClient serviceClient; + private final ObjectMapper smileMapper; + private final JavaType inMessageBatchType; + + public MessageRelayClientImpl( + final ServiceClient serviceClient, + final ObjectMapper smileMapper, + final Class inMessageClass + ) + { + this.serviceClient = serviceClient; + this.smileMapper = smileMapper; + this.inMessageBatchType = smileMapper.getTypeFactory().constructParametricType(MessageBatch.class, inMessageClass); + } + + @Override + public ListenableFuture> getMessages( + final String clientHost, + final long epoch, + final long startWatermark + ) + { + final String path = StringUtils.format( + "/outbox/%s/messages?epoch=%d&watermark=%d", + StringUtils.urlEncode(clientHost), + epoch, + startWatermark + ); + + return FutureUtils.transform( + serviceClient.asyncRequest( + new RequestBuilder(HttpMethod.GET, path), + new BytesFullResponseHandler() + ), + holder -> { + if (holder.getResponse().getStatus().getCode() == HttpStatus.NO_CONTENT_204) { + return new MessageBatch<>(Collections.emptyList(), epoch, startWatermark); + } else { + return JacksonUtils.readValue(smileMapper, holder.getContent(), inMessageBatchType); + } + } + ); + } +} diff --git a/server/src/main/java/org/apache/druid/messages/client/MessageRelayFactory.java b/server/src/main/java/org/apache/druid/messages/client/MessageRelayFactory.java new file mode 100644 index 000000000000..b647b9e4b6a2 --- /dev/null +++ b/server/src/main/java/org/apache/druid/messages/client/MessageRelayFactory.java @@ -0,0 +1,30 @@ +/* + * 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.messages.client; + +import org.apache.druid.server.DruidNode; + +/** + * Factory for creating new message relays. Used by {@link MessageRelays}. + */ +public interface MessageRelayFactory +{ + MessageRelay newRelay(DruidNode druidNode); +} diff --git a/server/src/main/java/org/apache/druid/messages/client/MessageRelayFactoryImpl.java b/server/src/main/java/org/apache/druid/messages/client/MessageRelayFactoryImpl.java new file mode 100644 index 000000000000..0d5c30d034d1 --- /dev/null +++ b/server/src/main/java/org/apache/druid/messages/client/MessageRelayFactoryImpl.java @@ -0,0 +1,76 @@ +/* + * 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.messages.client; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.rpc.FixedServiceLocator; +import org.apache.druid.rpc.ServiceClient; +import org.apache.druid.rpc.ServiceClientFactory; +import org.apache.druid.rpc.ServiceLocation; +import org.apache.druid.rpc.StandardRetryPolicy; +import org.apache.druid.server.DruidNode; + +/** + * Production implementation of {@link MessageRelayFactory}. + */ +public class MessageRelayFactoryImpl implements MessageRelayFactory +{ + private final String clientHost; + private final MessageListener messageListener; + private final ServiceClientFactory clientFactory; + private final String basePath; + private final ObjectMapper smileMapper; + private final Class messageClass; + + public MessageRelayFactoryImpl( + final String clientHost, + final MessageListener messageListener, + final ServiceClientFactory clientFactory, + final String basePath, + final ObjectMapper smileMapper, + final Class messageClass + ) + { + this.clientHost = clientHost; + this.messageListener = messageListener; + this.clientFactory = clientFactory; + this.smileMapper = smileMapper; + this.messageClass = messageClass; + this.basePath = basePath; + } + + @Override + public MessageRelay newRelay(DruidNode clientNode) + { + final ServiceLocation location = ServiceLocation.fromDruidNode(clientNode).withBasePath(basePath); + final ServiceClient client = clientFactory.makeClient( + clientNode.getHostAndPortToUse(), + new FixedServiceLocator(location), + StandardRetryPolicy.unlimited() + ); + + return new MessageRelay<>( + clientHost, + clientNode, + new MessageRelayClientImpl<>(client, smileMapper, messageClass), + messageListener + ); + } +} diff --git a/server/src/main/java/org/apache/druid/messages/client/MessageRelays.java b/server/src/main/java/org/apache/druid/messages/client/MessageRelays.java new file mode 100644 index 000000000000..ef43c25cf6a0 --- /dev/null +++ b/server/src/main/java/org/apache/druid/messages/client/MessageRelays.java @@ -0,0 +1,146 @@ +/* + * 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.messages.client; + +import com.google.errorprone.annotations.concurrent.GuardedBy; +import org.apache.druid.discovery.DiscoveryDruidNode; +import org.apache.druid.discovery.DruidNodeDiscovery; +import org.apache.druid.discovery.DruidNodeDiscoveryProvider; +import org.apache.druid.discovery.NodeRole; +import org.apache.druid.guice.ManageLifecycle; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.lifecycle.LifecycleStart; +import org.apache.druid.java.util.common.lifecycle.LifecycleStop; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.server.DruidNode; +import org.apache.druid.utils.CloseableUtils; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Manages a fleet of {@link MessageRelay}, one for each server discovered by a {@link DruidNodeDiscoveryProvider}. + */ +@ManageLifecycle +public class MessageRelays +{ + private static final Logger log = new Logger(MessageRelays.class); + + @GuardedBy("serverRelays") + private final Map> serverRelays = new HashMap<>(); + private final DruidNodeDiscoveryProvider discoveryProvider; + private final MessageRelayFactory messageRelayFactory; + private final NodeRole nodeRole; + private final Listener listener; + + private volatile DruidNodeDiscovery discovery; + + public MessageRelays( + final DruidNodeDiscoveryProvider discoveryProvider, + final MessageRelayFactory messageRelayFactory, + final NodeRole nodeRole + ) + { + this.discoveryProvider = discoveryProvider; + this.messageRelayFactory = messageRelayFactory; + this.nodeRole = nodeRole; + this.listener = new Listener(); + } + + @LifecycleStart + public void start() + { + discovery = discoveryProvider.getForNodeRole(nodeRole); + discovery.registerListener(listener); + } + + @LifecycleStop + public void stop() + { + if (discovery != null) { + discovery.removeListener(listener); + discovery = null; + } + + synchronized (serverRelays) { + try { + CloseableUtils.closeAll(serverRelays.values()); + } + catch (IOException e) { + throw new RuntimeException(e); + } + finally { + serverRelays.clear(); + } + } + } + + /** + * Discovery listener. Creates and tears down individual host relays. + */ + class Listener implements DruidNodeDiscovery.Listener + { + @Override + public void nodesAdded(final Collection nodes) + { + synchronized (serverRelays) { + for (final DiscoveryDruidNode node : nodes) { + final DruidNode druidNode = node.getDruidNode(); + + serverRelays.computeIfAbsent(druidNode.getHostAndPortToUse(), ignored -> { + final MessageRelay relay = messageRelayFactory.newRelay(druidNode); + relay.start(); + return relay; + }); + } + } + } + + @Override + public void nodesRemoved(final Collection nodes) + { + final List>> removed = new ArrayList<>(); + + synchronized (serverRelays) { + for (final DiscoveryDruidNode node : nodes) { + final DruidNode druidNode = node.getDruidNode(); + final String druidHost = druidNode.getHostAndPortToUse(); + final MessageRelay relay = serverRelays.remove(druidHost); + if (relay != null) { + removed.add(Pair.of(druidHost, relay)); + } + } + } + + for (final Pair> relay : removed) { + try { + relay.rhs.close(); + } + catch (Throwable e) { + log.noStackTrace().warn(e, "Could not close relay for server[%s]. Dropping.", relay.lhs); + } + } + } + } +} diff --git a/server/src/main/java/org/apache/druid/messages/server/ClientMonitor.java b/server/src/main/java/org/apache/druid/messages/server/ClientMonitor.java new file mode 100644 index 000000000000..6a537c9476d5 --- /dev/null +++ b/server/src/main/java/org/apache/druid/messages/server/ClientMonitor.java @@ -0,0 +1,82 @@ +/* + * 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.messages.server; + +import org.apache.druid.discovery.DiscoveryDruidNode; +import org.apache.druid.discovery.DruidNodeDiscovery; +import org.apache.druid.discovery.DruidNodeDiscoveryProvider; +import org.apache.druid.discovery.NodeRole; +import org.apache.druid.java.util.common.lifecycle.LifecycleStart; + +import java.util.Collection; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Code that runs on message servers, to monitor their clients. When a client vanishes, its outbox is reset using + * {@link Outbox#resetOutbox(String)}. + */ +public class ClientMonitor +{ + private final DruidNodeDiscoveryProvider discoveryProvider; + private final Outbox outbox; + private final NodeRole clientRole; + + public ClientMonitor( + final DruidNodeDiscoveryProvider discoveryProvider, + final Outbox outbox, + final NodeRole clientRole + ) + { + this.discoveryProvider = discoveryProvider; + this.outbox = outbox; + this.clientRole = clientRole; + } + + @LifecycleStart + public void start() + { + discoveryProvider.getForNodeRole(clientRole).registerListener(new ClientListener()); + } + + /** + * Listener that cancels work associated with clients that have gone away. + */ + private class ClientListener implements DruidNodeDiscovery.Listener + { + @Override + public void nodesAdded(Collection nodes) + { + // Nothing to do. Although, perhaps it would make sense to *set up* an outbox here. (Currently, outboxes are + // created on-demand as they receive messages.) + } + + @Override + public void nodesRemoved(Collection nodes) + { + final Set hostsRemoved = + nodes.stream().map(node -> node.getDruidNode().getHostAndPortToUse()).collect(Collectors.toSet()); + + for (final String clientHost : hostsRemoved) { + outbox.resetOutbox(clientHost); + } + } + } +} diff --git a/server/src/main/java/org/apache/druid/messages/server/MessageRelayResource.java b/server/src/main/java/org/apache/druid/messages/server/MessageRelayResource.java new file mode 100644 index 000000000000..f8e771d378c7 --- /dev/null +++ b/server/src/main/java/org/apache/druid/messages/server/MessageRelayResource.java @@ -0,0 +1,196 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.messages.server; + +import com.fasterxml.jackson.databind.JavaType; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.messages.MessageBatch; +import org.apache.druid.messages.client.MessageListener; +import org.apache.druid.messages.client.MessageRelayClient; + +import javax.servlet.AsyncContext; +import javax.servlet.AsyncEvent; +import javax.servlet.AsyncListener; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; +import javax.ws.rs.GET; +import javax.ws.rs.Path; +import javax.ws.rs.PathParam; +import javax.ws.rs.QueryParam; +import javax.ws.rs.core.Context; +import java.io.IOException; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * Server-side resource for message relaying. Wraps an {@link Outbox} and {@link MessageListener}. + * The client for this resource is {@link MessageRelayClient}. + */ +public class MessageRelayResource +{ + private static final Logger log = new Logger(MessageRelayResource.class); + private static final long GET_MESSAGES_TIMEOUT = 30_000L; + + /** + * Outbox for messages sent from this server. + */ + private final Outbox outbox; + + /** + * Message relay protocol uses Smile. + */ + private final ObjectMapper smileMapper; + + /** + * Type of {@link MessageBatch} of {@link MessageType}. + */ + private final JavaType batchType; + + public MessageRelayResource( + final Outbox outbox, + final ObjectMapper smileMapper, + final Class messageClass + ) + { + this.outbox = outbox; + this.smileMapper = smileMapper; + this.batchType = smileMapper.getTypeFactory().constructParametricType(MessageBatch.class, messageClass); + } + + /** + * Retrieve messages from the outbox for a particular client, as a {@link MessageBatch} in Smile format. + * The messages are retrieved from {@link Outbox#getMessages(String, long, long)}. + * + * This is a long-polling async method, using {@link AsyncContext} to wait up to {@link #GET_MESSAGES_TIMEOUT} for + * messages to appear in the outbox. + * + * @return HTTP 200 with Smile response with messages on success; HTTP 204 (No Content) if no messages were put in + * the outbox before the timeout {@link #GET_MESSAGES_TIMEOUT} elapsed + * + * @see Outbox#getMessages(String, long, long) for more details on the API + */ + @GET + @Path("/outbox/{clientHost}/messages") + public Void httpGetMessagesFromOutbox( + @PathParam("clientHost") final String clientHost, + @QueryParam("epoch") final Long epoch, + @QueryParam("watermark") final Long watermark, + @Context final HttpServletRequest req + ) throws IOException + { + if (epoch == null || watermark == null || clientHost == null || clientHost.isEmpty()) { + AsyncContext asyncContext = req.startAsync(); + HttpServletResponse response = (HttpServletResponse) asyncContext.getResponse(); + response.sendError(HttpServletResponse.SC_BAD_REQUEST); + asyncContext.complete(); + return null; + } + + final AtomicBoolean didRespond = new AtomicBoolean(); + final ListenableFuture> batchFuture = outbox.getMessages(clientHost, epoch, watermark); + final AsyncContext asyncContext = req.startAsync(); + asyncContext.setTimeout(GET_MESSAGES_TIMEOUT); + asyncContext.addListener( + new AsyncListener() + { + @Override + public void onComplete(AsyncEvent event) + { + } + + @Override + public void onTimeout(AsyncEvent event) + { + if (didRespond.compareAndSet(false, true)) { + HttpServletResponse response = (HttpServletResponse) asyncContext.getResponse(); + response.setStatus(HttpServletResponse.SC_NO_CONTENT); + event.getAsyncContext().complete(); + batchFuture.cancel(true); + } + } + + @Override + public void onError(AsyncEvent event) + { + } + + @Override + public void onStartAsync(AsyncEvent event) + { + } + } + ); + + // Save these items, since "req" becomes inaccessible in future exception handlers. + final String remoteAddr = req.getRemoteAddr(); + final String requestURI = req.getRequestURI(); + + Futures.addCallback( + batchFuture, + new FutureCallback>() + { + @Override + public void onSuccess(MessageBatch result) + { + if (didRespond.compareAndSet(false, true)) { + log.debug("Sending message batch: %s", result); + try { + HttpServletResponse response = (HttpServletResponse) asyncContext.getResponse(); + response.setStatus(HttpServletResponse.SC_OK); + response.setContentType(SmileMediaTypes.APPLICATION_JACKSON_SMILE); + smileMapper.writerFor(batchType) + .writeValue(asyncContext.getResponse().getOutputStream(), result); + response.getOutputStream().close(); + asyncContext.complete(); + } + catch (Exception e) { + log.noStackTrace().warn(e, "Could not respond to request from[%s] to[%s]", remoteAddr, requestURI); + } + } + } + + @Override + public void onFailure(Throwable e) + { + if (didRespond.compareAndSet(false, true)) { + try { + HttpServletResponse response = (HttpServletResponse) asyncContext.getResponse(); + response.sendError(HttpServletResponse.SC_INTERNAL_SERVER_ERROR); + asyncContext.complete(); + } + catch (Exception e2) { + e.addSuppressed(e2); + } + + log.noStackTrace().warn(e, "Request failed from[%s] to[%s]", remoteAddr, requestURI); + } + } + }, + Execs.directExecutor() + ); + + return null; + } +} diff --git a/server/src/main/java/org/apache/druid/messages/server/Outbox.java b/server/src/main/java/org/apache/druid/messages/server/Outbox.java new file mode 100644 index 000000000000..529f3e8fd337 --- /dev/null +++ b/server/src/main/java/org/apache/druid/messages/server/Outbox.java @@ -0,0 +1,66 @@ +/* + * 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.messages.server; + +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.messages.MessageBatch; +import org.apache.druid.messages.client.MessageRelay; + +/** + * An outbox for messages sent from servers to clients. + */ +public interface Outbox +{ + /** + * Send a message to a client, through an outbox. + * + * @param clientHost which outbox to send messages through. Each clientHost has its own outbox. + * @param message message to send + * + * @return future that resolves successfully when the client has acknowledged the message + */ + ListenableFuture sendMessage(String clientHost, MessageType message); + + /** + * Get the next batch of messages for an client, from an outbox. + * + * The provided epoch must either be {@link MessageRelay#INIT}, or must match the epoch of the outbox as indicated by + * {@link MessageBatch#getEpoch()} returned by previous calls to the same outbox. If the provided epoch does not + * match, an empty batch is returned with the correct epoch indicated in {@link MessageBatch#getEpoch()}. + * + * The provided watermark must be greater than, or equal to, the previous watermark supplied to the same outbox. + * Any messages lower than the watermark are acknowledged and removed from the outbox. + * + * @param clientHost which outbox to retrieve messages from. Each clientHost has its own outbox. + * @param epoch outbox epoch, or {@link MessageRelay#INIT} if this is the first call from the collector. + * @param startWatermark outbox message watermark to retrieve from. + * + * @return future that resolves to the next batch of messages + */ + ListenableFuture> getMessages(String clientHost, long epoch, long startWatermark); + + /** + * Reset the outbox for a particular client. This removes all messages, cancels all outstanding futures, and + * resets the epoch. + * + * @param clientHost the client host:port + */ + void resetOutbox(final String clientHost); +} diff --git a/server/src/main/java/org/apache/druid/messages/server/OutboxImpl.java b/server/src/main/java/org/apache/druid/messages/server/OutboxImpl.java new file mode 100644 index 000000000000..6f21601ad00a --- /dev/null +++ b/server/src/main/java/org/apache/druid/messages/server/OutboxImpl.java @@ -0,0 +1,218 @@ +/* + * 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.messages.server; + +import com.google.common.collect.Sets; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.SettableFuture; +import com.google.errorprone.annotations.concurrent.GuardedBy; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.lifecycle.LifecycleStop; +import org.apache.druid.messages.MessageBatch; +import org.apache.druid.messages.client.MessageRelay; + +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Deque; +import java.util.Iterator; +import java.util.List; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ThreadLocalRandom; + +/** + * Production implementation of {@link Outbox}. Each outbox is represented by an {@link OutboxQueue}. + */ +public class OutboxImpl implements Outbox +{ + private static final int MAX_BATCH_SIZE = 8; + + // clientHost -> outgoing message queue + private final ConcurrentHashMap> queues; + private volatile boolean stopped; + + public OutboxImpl() + { + this.queues = new ConcurrentHashMap<>(); + } + + @LifecycleStop + public void stop() + { + stopped = true; + + final Iterator> it = queues.values().iterator(); + while (it.hasNext()) { + it.next().stop(); + it.remove(); + } + } + + @Override + public ListenableFuture sendMessage(String clientHost, MessageType message) + { + if (stopped) { + return Futures.immediateCancelledFuture(); + } + + return queues.computeIfAbsent(clientHost, id -> new OutboxQueue<>()) + .sendMessage(message); + } + + @Override + public ListenableFuture> getMessages(String clientHost, long epoch, long startWatermark) + { + if (stopped) { + return Futures.immediateCancelledFuture(); + } + + final OutboxQueue queue = queues.computeIfAbsent(clientHost, id -> new OutboxQueue<>()); + if (epoch != queue.epoch && epoch != MessageRelay.INIT) { + return Futures.immediateFuture(new MessageBatch<>(Collections.emptyList(), queue.epoch, 0)); + } + + return queue.getMessages(startWatermark); + } + + @Override + public void resetOutbox(final String clientHost) + { + final OutboxQueue queue = queues.remove(clientHost); + if (queue != null) { + queue.stop(); + } + } + + /** + * Outgoing queue for a specific client. + */ + public static class OutboxQueue + { + /** + * Epoch, set when the outbox is created. Attached to returned batches through {@link MessageBatch#getEpoch()}. + */ + private final long epoch; + + /** + * Currently-outstanding futures. These are tracked so they can be canceled in {@link #stop()}. + */ + private final Set> pendingFutures = Sets.newConcurrentHashSet(); + + @GuardedBy("this") + private long startWatermark = 0; + + @GuardedBy("this") + private final Deque, T>> queue = new ArrayDeque<>(); + + @GuardedBy("this") + private SettableFuture messageAvailableFuture = SettableFuture.create(); + + private volatile boolean stopped; + + public OutboxQueue() + { + this.epoch = ThreadLocalRandom.current().nextLong() & Long.MAX_VALUE; + } + + ListenableFuture sendMessage(final T message) + { + final SettableFuture future = SettableFuture.create(); + + synchronized (this) { + queue.add(Pair.of(future, message)); + if (!messageAvailableFuture.isDone()) { + messageAvailableFuture.set(null); + } + } + + return registerFuture(future); + } + + ListenableFuture> getMessages(final long newStartWatermark) + { + synchronized (this) { + // Ack and drain all messages up to startWatermark. + while (!queue.isEmpty() && startWatermark < newStartWatermark) { + final Pair, T> message = queue.poll(); + startWatermark++; + message.lhs.set(null); + } + + if (queue.isEmpty()) { + // Send next batch when a message is available. + if (messageAvailableFuture.isDone()) { + messageAvailableFuture = SettableFuture.create(); + } + + return registerFuture( + FutureUtils.transform( + Futures.nonCancellationPropagating(messageAvailableFuture), + ignored -> { + synchronized (this) { + return nextBatch(); + } + } + ) + ); + } else { + return registerFuture(Futures.immediateFuture(nextBatch())); + } + } + } + + void stop() + { + stopped = true; + for (ListenableFuture future : pendingFutures) { + future.cancel(false); // Ignore return value + } + } + + @GuardedBy("this") + private MessageBatch nextBatch() + { + final List batch = new ArrayList<>(); + final Iterator, T>> it = queue.iterator(); + + while (it.hasNext() && batch.size() < MAX_BATCH_SIZE) { + batch.add(it.next().rhs); + } + + return new MessageBatch<>(batch, epoch, startWatermark); + } + + private ListenableFuture registerFuture(final ListenableFuture future) + { + pendingFutures.add(future); + future.addListener(() -> pendingFutures.remove(future), Execs.directExecutor()); + + // If "stop" was called while we were creating this future, cancel it prior to returning it. + if (stopped) { + future.cancel(false); + } + + return future; + } + } +} diff --git a/server/src/main/java/org/apache/druid/rpc/FixedServiceLocator.java b/server/src/main/java/org/apache/druid/rpc/FixedServiceLocator.java new file mode 100644 index 000000000000..d2ed27b24eb1 --- /dev/null +++ b/server/src/main/java/org/apache/druid/rpc/FixedServiceLocator.java @@ -0,0 +1,59 @@ +/* + * 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; + +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; + +/** + * Locator for a fixed set of {@link ServiceLocations}. + */ +public class FixedServiceLocator implements ServiceLocator +{ + private final ServiceLocations locations; + + private volatile boolean closed = false; + + public FixedServiceLocator(final ServiceLocations locations) + { + this.locations = locations; + } + + public FixedServiceLocator(final ServiceLocation location) + { + this(ServiceLocations.forLocation(location)); + } + + @Override + public ListenableFuture locate() + { + if (closed) { + return Futures.immediateFuture(ServiceLocations.closed()); + } else { + return Futures.immediateFuture(locations); + } + } + + @Override + public void close() + { + closed = true; + } +} diff --git a/server/src/main/java/org/apache/druid/rpc/ServiceClientImpl.java b/server/src/main/java/org/apache/druid/rpc/ServiceClientImpl.java index 3178360016ab..172f220fabad 100644 --- a/server/src/main/java/org/apache/druid/rpc/ServiceClientImpl.java +++ b/server/src/main/java/org/apache/druid/rpc/ServiceClientImpl.java @@ -41,7 +41,6 @@ import javax.annotation.Nullable; import java.net.URI; -import java.net.URISyntaxException; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; @@ -497,19 +496,7 @@ static long computeBackoffMs(final ServiceRetryPolicy retryPolicy, final long at } /** - * Sanitizes IPv6 address if it has brackets. Eg. host = "[1:2:3:4:5:6:7:8]" will be returned as "1:2:3:4:5:6:7:8" - * after this function - */ - static String sanitizeHost(String host) - { - if (host.charAt(0) == '[') { - return host.substring(1, host.length() - 1); - } - return host; - } - - /** - * Returns a {@link ServiceLocation} without a path component, based on a URI. + * Returns a {@link ServiceLocation} without a path component, based on a URI. Returns null on invalid URIs. */ @Nullable @VisibleForTesting @@ -520,24 +507,17 @@ static ServiceLocation serviceLocationNoPathFromUri(@Nullable final String uriSt } try { - final URI uri = new URI(uriString); - - if (uri.getHost() == null) { - return null; - } - - final String scheme = uri.getScheme(); - final String host = sanitizeHost(uri.getHost()); - - if ("http".equals(scheme)) { - return new ServiceLocation(host, uri.getPort() < 0 ? 80 : uri.getPort(), -1, ""); - } else if ("https".equals(scheme)) { - return new ServiceLocation(host, -1, uri.getPort() < 0 ? 443 : uri.getPort(), ""); - } else { - return null; - } + final ServiceLocation location = ServiceLocation.fromUri(URI.create(uriString)); + + // Strip path. + return new ServiceLocation( + location.getHost(), + location.getPlaintextPort(), + location.getTlsPort(), + "" + ); } - catch (URISyntaxException e) { + catch (IllegalArgumentException e) { return null; } } @@ -549,8 +529,8 @@ static ServiceLocation serviceLocationNoPathFromUri(@Nullable final String uriSt static boolean serviceLocationMatches(final ServiceLocation left, final ServiceLocation right) { return left.getHost().equals(right.getHost()) - && portMatches(left.getPlaintextPort(), right.getPlaintextPort()) - && portMatches(left.getTlsPort(), right.getTlsPort()); + && portMatches(left.getPlaintextPort(), right.getPlaintextPort()) + && portMatches(left.getTlsPort(), right.getTlsPort()); } static boolean portMatches(int left, int right) diff --git a/server/src/main/java/org/apache/druid/rpc/ServiceLocation.java b/server/src/main/java/org/apache/druid/rpc/ServiceLocation.java index aeaa24318e93..974f09fe89bb 100644 --- a/server/src/main/java/org/apache/druid/rpc/ServiceLocation.java +++ b/server/src/main/java/org/apache/druid/rpc/ServiceLocation.java @@ -22,6 +22,7 @@ import com.google.common.base.Preconditions; import com.google.common.base.Splitter; import com.google.common.collect.ImmutableList; +import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordination.DruidServerMetadata; @@ -29,6 +30,7 @@ import javax.annotation.Nullable; import javax.validation.constraints.NotNull; import java.net.MalformedURLException; +import java.net.URI; import java.net.URL; import java.util.Iterator; import java.util.Objects; @@ -40,6 +42,8 @@ public class ServiceLocation { private static final String HTTP_SCHEME = "http"; private static final String HTTPS_SCHEME = "https"; + private static final int HTTP_DEFAULT_PORT = 80; + private static final int HTTPS_DEFAULT_PORT = 443; private static final Splitter HOST_SPLITTER = Splitter.on(":").limit(2); private final String host; @@ -72,6 +76,50 @@ public static ServiceLocation fromDruidNode(final DruidNode druidNode) return new ServiceLocation(druidNode.getHost(), druidNode.getPlaintextPort(), druidNode.getTlsPort(), ""); } + /** + * Create a service location based on a {@link URI}. + * + * @throws IllegalArgumentException if the URI cannot be mapped to a service location. + */ + public static ServiceLocation fromUri(final URI uri) + { + if (uri == null || uri.getHost() == null) { + throw new IAE("URI[%s] has no host", uri); + } + + final String scheme = uri.getScheme(); + final String host = stripBrackets(uri.getHost()); + final StringBuilder basePath = new StringBuilder(); + + if (uri.getRawPath() != null) { + if (uri.getRawQuery() == null && uri.getRawFragment() == null && uri.getRawPath().endsWith("/")) { + // Strip trailing slash if the URI has no query or fragment. By convention, this trailing slash is not + // part of the service location. + basePath.append(uri.getRawPath(), 0, uri.getRawPath().length() - 1); + } else { + basePath.append(uri.getRawPath()); + } + } + + if (uri.getRawQuery() != null) { + basePath.append('?').append(uri.getRawQuery()); + } + + if (uri.getRawFragment() != null) { + basePath.append('#').append(uri.getRawFragment()); + } + + if (HTTP_SCHEME.equals(scheme)) { + final int port = uri.getPort() < 0 ? HTTP_DEFAULT_PORT : uri.getPort(); + return new ServiceLocation(host, port, -1, basePath.toString()); + } else if (HTTPS_SCHEME.equals(scheme)) { + final int port = uri.getPort() < 0 ? HTTPS_DEFAULT_PORT : uri.getPort(); + return new ServiceLocation(host, -1, port, basePath.toString()); + } else { + throw new IAE("URI[%s] has invalid scheme[%s]", uri, scheme); + } + } + /** * Create a service location based on a {@link DruidServerMetadata}. * @@ -133,6 +181,11 @@ public String getBasePath() return basePath; } + public ServiceLocation withBasePath(final String newBasePath) + { + return new ServiceLocation(host, plaintextPort, tlsPort, newBasePath); + } + public URL toURL(@Nullable final String encodedPathAndQueryString) { final String scheme; @@ -193,4 +246,15 @@ public String toString() '}'; } + /** + * Strips brackers from the host part of a URI, so we can better handle IPv6 addresses. + * e.g. host = "[1:2:3:4:5:6:7:8]" is transformed to "1:2:3:4:5:6:7:8" by this function + */ + static String stripBrackets(String host) + { + if (host.charAt(0) == '[' && host.charAt(host.length() - 1) == ']') { + return host.substring(1, host.length() - 1); + } + return host; + } } diff --git a/server/src/test/java/org/apache/druid/rpc/ServiceClientImplTest.java b/server/src/test/java/org/apache/druid/rpc/ServiceClientImplTest.java index 69cb12e423ca..7346edd5cf6b 100644 --- a/server/src/test/java/org/apache/druid/rpc/ServiceClientImplTest.java +++ b/server/src/test/java/org/apache/druid/rpc/ServiceClientImplTest.java @@ -685,14 +685,6 @@ public void test_serviceLocationNoPathFromUri() ); } - @Test - public void test_normalizeHost() - { - Assert.assertEquals("1:2:3:4:5:6:7:8", ServiceClientImpl.sanitizeHost("[1:2:3:4:5:6:7:8]")); - Assert.assertEquals("1:2:3:4:5:6:7:8", ServiceClientImpl.sanitizeHost("1:2:3:4:5:6:7:8")); - Assert.assertEquals("1.2.3.4", ServiceClientImpl.sanitizeHost("1.2.3.4")); - } - @Test public void test_isRedirect() { diff --git a/server/src/test/java/org/apache/druid/rpc/ServiceLocationTest.java b/server/src/test/java/org/apache/druid/rpc/ServiceLocationTest.java index 6aec0e2b6060..6594fbbf0548 100644 --- a/server/src/test/java/org/apache/druid/rpc/ServiceLocationTest.java +++ b/server/src/test/java/org/apache/druid/rpc/ServiceLocationTest.java @@ -27,6 +27,14 @@ public class ServiceLocationTest { + @Test + public void test_stripBrackets() + { + Assert.assertEquals("1:2:3:4:5:6:7:8", ServiceLocation.stripBrackets("[1:2:3:4:5:6:7:8]")); + Assert.assertEquals("1:2:3:4:5:6:7:8", ServiceLocation.stripBrackets("1:2:3:4:5:6:7:8")); + Assert.assertEquals("1.2.3.4", ServiceLocation.stripBrackets("1.2.3.4")); + } + @Test public void test_fromDruidServerMetadata_withPort() { diff --git a/services/src/main/java/org/apache/druid/cli/CliHistorical.java b/services/src/main/java/org/apache/druid/cli/CliHistorical.java index 2e231bcdcc3b..ea8bbd994348 100644 --- a/services/src/main/java/org/apache/druid/cli/CliHistorical.java +++ b/services/src/main/java/org/apache/druid/cli/CliHistorical.java @@ -42,6 +42,7 @@ import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.guice.QueryRunnerFactoryModule; import org.apache.druid.guice.QueryableModule; +import org.apache.druid.guice.SegmentWranglerModule; import org.apache.druid.guice.ServerTypeConfig; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.QuerySegmentWalker; @@ -99,6 +100,7 @@ protected List getModules() new DruidProcessingModule(), new QueryableModule(), new QueryRunnerFactoryModule(), + new SegmentWranglerModule(), new JoinableFactoryModule(), new HistoricalServiceModule(), binder -> { diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/IngestHandler.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/IngestHandler.java index e0b5ffdb08e3..3569864cae78 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/IngestHandler.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/IngestHandler.java @@ -186,7 +186,8 @@ protected RelDataType returnedRowType() final RelDataTypeFactory typeFactory = rootQueryRel.rel.getCluster().getTypeFactory(); return handlerContext.engine().resultTypeForInsert( typeFactory, - rootQueryRel.validatedRowType + rootQueryRel.validatedRowType, + handlerContext.queryContextMap() ); } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryHandler.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryHandler.java index a915833cd3ff..b19e83e040e0 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryHandler.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryHandler.java @@ -726,7 +726,8 @@ protected RelDataType returnedRowType() final RelDataTypeFactory typeFactory = rootQueryRel.rel.getCluster().getTypeFactory(); return handlerContext.engine().resultTypeForSelect( typeFactory, - rootQueryRel.validatedRowType + rootQueryRel.validatedRowType, + handlerContext.plannerContext().queryContextMap() ); } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/run/NativeSqlEngine.java b/sql/src/main/java/org/apache/druid/sql/calcite/run/NativeSqlEngine.java index d02d302437b8..4f3d86b1b420 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/run/NativeSqlEngine.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/run/NativeSqlEngine.java @@ -83,13 +83,21 @@ public void validateContext(Map queryContext) } @Override - public RelDataType resultTypeForSelect(RelDataTypeFactory typeFactory, RelDataType validatedRowType) + public RelDataType resultTypeForSelect( + RelDataTypeFactory typeFactory, + RelDataType validatedRowType, + Map queryContext + ) { return validatedRowType; } @Override - public RelDataType resultTypeForInsert(RelDataTypeFactory typeFactory, RelDataType validatedRowType) + public RelDataType resultTypeForInsert( + RelDataTypeFactory typeFactory, + RelDataType validatedRowType, + Map queryContext + ) { throw new UnsupportedOperationException(); } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/run/SqlEngine.java b/sql/src/main/java/org/apache/druid/sql/calcite/run/SqlEngine.java index fec7660e44ef..1d33b019e684 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/run/SqlEngine.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/run/SqlEngine.java @@ -57,8 +57,13 @@ public interface SqlEngine * * @param typeFactory type factory * @param validatedRowType row type from Calcite's validator + * @param queryContext query context, in case that affects the result type */ - RelDataType resultTypeForSelect(RelDataTypeFactory typeFactory, RelDataType validatedRowType); + RelDataType resultTypeForSelect( + RelDataTypeFactory typeFactory, + RelDataType validatedRowType, + Map queryContext + ); /** * SQL row type that would be emitted by the {@link QueryMaker} from {@link #buildQueryMakerForInsert}. @@ -66,8 +71,13 @@ public interface SqlEngine * * @param typeFactory type factory * @param validatedRowType row type from Calcite's validator + * @param queryContext query context, in case that affects the result type */ - RelDataType resultTypeForInsert(RelDataTypeFactory typeFactory, RelDataType validatedRowType); + RelDataType resultTypeForInsert( + RelDataTypeFactory typeFactory, + RelDataType validatedRowType, + Map queryContext + ); /** * Create a {@link QueryMaker} for a SELECT query. diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/view/ViewSqlEngine.java b/sql/src/main/java/org/apache/druid/sql/calcite/view/ViewSqlEngine.java index 716fa50b85f1..7563b45d52bc 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/view/ViewSqlEngine.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/view/ViewSqlEngine.java @@ -93,13 +93,21 @@ public void validateContext(Map queryContext) } @Override - public RelDataType resultTypeForSelect(RelDataTypeFactory typeFactory, RelDataType validatedRowType) + public RelDataType resultTypeForSelect( + RelDataTypeFactory typeFactory, + RelDataType validatedRowType, + Map queryContext + ) { return validatedRowType; } @Override - public RelDataType resultTypeForInsert(RelDataTypeFactory typeFactory, RelDataType validatedRowType) + public RelDataType resultTypeForInsert( + RelDataTypeFactory typeFactory, + RelDataType validatedRowType, + Map queryContext + ) { // Can't have views of INSERT or REPLACE statements. throw new UnsupportedOperationException(); diff --git a/sql/src/main/java/org/apache/druid/sql/http/SqlResource.java b/sql/src/main/java/org/apache/druid/sql/http/SqlResource.java index 4adea5d8d84e..d957e7155b5e 100644 --- a/sql/src/main/java/org/apache/druid/sql/http/SqlResource.java +++ b/sql/src/main/java/org/apache/druid/sql/http/SqlResource.java @@ -82,7 +82,7 @@ public class SqlResource private final DruidNode selfNode; @Inject - SqlResource( + protected SqlResource( final ObjectMapper jsonMapper, final AuthorizerMapper authorizerMapper, final @NativeQuery SqlStatementFactory sqlStatementFactory, @@ -140,19 +140,7 @@ public Response cancelQuery( return Response.status(Status.NOT_FOUND).build(); } - // Considers only datasource and table resources; not context - // key resources when checking permissions. This means that a user's - // permission to cancel a query depends on the datasource, not the - // context variables used in the query. - Set resources = lifecycles - .stream() - .flatMap(lifecycle -> lifecycle.resources().stream()) - .collect(Collectors.toSet()); - Access access = AuthorizationUtils.authorizeAllResourceActions( - req, - resources, - authorizerMapper - ); + final Access access = authorizeCancellation(req, lifecycles); if (access.isAllowed()) { // should remove only the lifecycles in the snapshot. @@ -341,4 +329,23 @@ public void writeException(Exception ex, OutputStream out) throws IOException out.write(jsonMapper.writeValueAsBytes(ex)); } } + + /** + * Authorize a query cancellation operation. + * + * Considers only datasource and table resources; not context key resources when checking permissions. This means + * that a user's permission to cancel a query depends on the datasource, not the context variables used in the query. + */ + public Access authorizeCancellation(final HttpServletRequest req, final List cancelables) + { + Set resources = cancelables + .stream() + .flatMap(lifecycle -> lifecycle.resources().stream()) + .collect(Collectors.toSet()); + return AuthorizationUtils.authorizeAllResourceActions( + req, + resources, + authorizerMapper + ); + } } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteScanSignatureTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteScanSignatureTest.java index abab053dd6bb..80a9dde9b4c9 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteScanSignatureTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteScanSignatureTest.java @@ -155,13 +155,21 @@ public void validateContext(Map queryContext) } @Override - public RelDataType resultTypeForSelect(RelDataTypeFactory typeFactory, RelDataType validatedRowType) + public RelDataType resultTypeForSelect( + RelDataTypeFactory typeFactory, + RelDataType validatedRowType, + Map queryContext + ) { return validatedRowType; } @Override - public RelDataType resultTypeForInsert(RelDataTypeFactory typeFactory, RelDataType validatedRowType) + public RelDataType resultTypeForInsert( + RelDataTypeFactory typeFactory, + RelDataType validatedRowType, + Map queryContext + ) { throw new UnsupportedOperationException(); } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/IngestionTestSqlEngine.java b/sql/src/test/java/org/apache/druid/sql/calcite/IngestionTestSqlEngine.java index 466bd0e390bd..569598af1e4e 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/IngestionTestSqlEngine.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/IngestionTestSqlEngine.java @@ -56,13 +56,21 @@ public void validateContext(Map queryContext) } @Override - public RelDataType resultTypeForSelect(RelDataTypeFactory typeFactory, RelDataType validatedRowType) + public RelDataType resultTypeForSelect( + RelDataTypeFactory typeFactory, + RelDataType validatedRowType, + Map queryContext + ) { throw new UnsupportedOperationException(); } @Override - public RelDataType resultTypeForInsert(RelDataTypeFactory typeFactory, RelDataType validatedRowType) + public RelDataType resultTypeForInsert( + RelDataTypeFactory typeFactory, + RelDataType validatedRowType, + Map queryContext + ) { // Matches the return structure of TestInsertQueryMaker. return typeFactory.createStructType( diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/TestTimelineServerView.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/TestTimelineServerView.java index bd80aee8cdad..58990e806617 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/TestTimelineServerView.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/TestTimelineServerView.java @@ -34,7 +34,6 @@ import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.TimelineLookup; -import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -97,7 +96,6 @@ public Optional> getTimeline(Da throw new UnsupportedOperationException(); } - @Nullable @Override public List getDruidServers() {