From 5f2db79bd9102d646d9fb0155af5f555313ba172 Mon Sep 17 00:00:00 2001 From: Sorabh Hamirwasia Date: Fri, 22 Mar 2019 16:43:25 -0700 Subject: [PATCH] Added stubs for QueryResourceManager exit and wait/cleanup thread Update MemoryCalculator to use DrillNode instead of DrillbitEndpoint Changes to support localbit resource registration to cluster state blob using DrillbitStatusListener Support ThrottledResourceManager via ResourceManagerBuilder Add some E2E tests and RMStateBlobs tests along with some bug fixes Fix TestRMConfigLoad tests to handle case where ZKQueues are explicitly enabled --- .../drill/yarn/zk/ZKClusterCoordinator.java | 64 ++- .../org/apache/drill/yarn/zk/ZKRegistry.java | 25 +- .../apache/drill/yarn/zk/TestZkRegistry.java | 27 +- .../drill/exec/coord/ClusterCoordinator.java | 11 +- .../exec/coord/zk/ZKClusterCoordinator.java | 45 +- .../apache/drill/exec/ops/QueryContext.java | 28 +- .../DistributedQueueParallelizer.java | 56 ++- .../planner/fragment/MemoryCalculator.java | 26 +- .../drill/exec/planner/fragment/Wrapper.java | 26 +- .../drill/exec/resourcemgr/NodeResources.java | 6 +- .../config/QueryQueueConfigImpl.java | 2 +- .../exec/resourcemgr/config/ResourcePool.java | 3 + .../resourcemgr/config/ResourcePoolImpl.java | 9 +- .../resourcemgr/config/ResourcePoolTree.java | 2 + .../config/ResourcePoolTreeImpl.java | 11 +- .../BestFitQueueSelection.java | 9 +- .../DefaultQueueSelection.java | 6 +- .../selectionpolicy/RandomQueueSelection.java | 4 +- .../rmblobmgr/RMBlobStoreManager.java | 10 +- .../RMConsistentBlobStoreManager.java | 195 ++++++-- .../rmblobmgr/rmblob/ClusterStateBlob.java | 1 + .../rmblob/ForemanResourceUsage.java | 1 + .../drill/exec/rpc/user/UserSession.java | 30 +- .../apache/drill/exec/server/Drillbit.java | 5 +- .../drill/exec/server/DrillbitContext.java | 35 +- .../work/foreman/DrillbitStatusListener.java | 12 +- .../drill/exec/work/foreman/Foreman.java | 18 +- .../drill/exec/work/foreman/QueryManager.java | 21 +- .../foreman/rm/DefaultResourceManager.java | 78 +-- .../rm/DistributedResourceManager.java | 444 +++++++++++++++--- .../foreman/rm/DynamicResourceManager.java | 5 - .../work/foreman/rm/QueryResourceManager.java | 69 ++- .../exec/work/foreman/rm/ResourceManager.java | 2 - .../foreman/rm/ResourceManagerBuilder.java | 16 +- .../foreman/rm/ThrottledResourceManager.java | 76 ++- .../exec/work/fragment/FragmentExecutor.java | 23 +- .../PhysicalPlanReaderTestFactory.java | 12 +- .../exec/planner/rm/TestMemoryCalculator.java | 45 +- .../drill/exec/pop/TestFragmentChecker.java | 11 +- .../exec/resourcemgr/RMBlobManagerTest.java | 354 ++++++++++---- .../resourcemgr/TestDistributedQueryRM.java | 156 ++++++ .../resourcemgr/TestE2EWithDistributedRM.java | 58 +++ .../exec/resourcemgr/TestRMConfigLoad.java | 30 +- .../org/apache/drill/test/ExampleTest.java | 17 +- .../apache/drill/exec/memory/Accountant.java | 12 +- 45 files changed, 1523 insertions(+), 573 deletions(-) create mode 100644 exec/java-exec/src/test/java/org/apache/drill/exec/resourcemgr/TestDistributedQueryRM.java create mode 100644 exec/java-exec/src/test/java/org/apache/drill/exec/resourcemgr/TestE2EWithDistributedRM.java diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/zk/ZKClusterCoordinator.java b/drill-yarn/src/main/java/org/apache/drill/yarn/zk/ZKClusterCoordinator.java index e79831278cc..27cd48fecce 100644 --- a/drill-yarn/src/main/java/org/apache/drill/yarn/zk/ZKClusterCoordinator.java +++ b/drill-yarn/src/main/java/org/apache/drill/yarn/zk/ZKClusterCoordinator.java @@ -17,17 +17,6 @@ */ package org.apache.drill.yarn.zk; -import static org.apache.drill.shaded.guava.com.google.common.collect.Collections2.transform; - -import java.io.IOException; -import java.util.Collection; -import java.util.Collections; -import java.util.HashSet; -import java.util.Set; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; - -import org.apache.drill.shaded.guava.com.google.common.base.Throwables; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.curator.RetryPolicy; @@ -55,8 +44,16 @@ import org.apache.drill.exec.coord.zk.ZkTransientStoreFactory; import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint; import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.State; +import org.apache.drill.shaded.guava.com.google.common.base.Throwables; -import org.apache.drill.shaded.guava.com.google.common.base.Function; +import java.io.IOException; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; /** * Manages cluster coordination utilizing zookeeper. @@ -87,8 +84,7 @@ public class ZKClusterCoordinator extends ClusterCoordinator { private CuratorFramework curator; private ServiceDiscovery discovery; - private volatile Collection endpoints = Collections - .emptyList(); + private volatile Map endpointsMap = Collections.emptyMap(); private final String serviceName; private final CountDownLatch initialConnection = new CountDownLatch(1); private final TransientStoreFactory factory; @@ -214,7 +210,7 @@ public void unregister(RegistrationHandle handle) { @Override public Collection getAvailableEndpoints() { - return this.endpoints; + return this.endpointsMap.values(); } @Override @@ -233,35 +229,33 @@ public TransientStore getOrCreateTransientStore( private synchronized void updateEndpoints() { try { - Collection newDrillbitSet = transform( - discovery.queryForInstances(serviceName), - new Function, DrillbitEndpoint>() { - @Override - public DrillbitEndpoint apply( - ServiceInstance input) { - return input.getPayload(); - } - }); + // All active bits in the Zookeeper + final Map UUIDtoEndpoints = discovery.queryForInstances(serviceName).stream() + .collect(Collectors.toConcurrentMap(ServiceInstance::getId, ServiceInstance::getPayload)); // set of newly dead bits : original bits - new set of active bits. - Set unregisteredBits = new HashSet<>(endpoints); - unregisteredBits.removeAll(newDrillbitSet); + Map unregisteredBits = new HashMap<>(endpointsMap); + for (Map.Entry newEndpoint : UUIDtoEndpoints.entrySet()) { + unregisteredBits.remove(newEndpoint.getKey()); + } // Set of newly live bits : new set of active bits - original bits. - Set registeredBits = new HashSet<>(newDrillbitSet); - registeredBits.removeAll(endpoints); + Map registeredBits = new HashMap<>(UUIDtoEndpoints); + for (Map.Entry newEndpoint : endpointsMap.entrySet()) { + registeredBits.remove(newEndpoint.getKey()); + } - endpoints = newDrillbitSet; + endpointsMap = UUIDtoEndpoints; if (logger.isDebugEnabled()) { StringBuilder builder = new StringBuilder(); builder.append("Active drillbit set changed. Now includes "); - builder.append(newDrillbitSet.size()); + builder.append(UUIDtoEndpoints.size()); builder.append(" total bits."); - if (!newDrillbitSet.isEmpty()) { + if (!UUIDtoEndpoints.isEmpty()) { builder.append(" New active drillbits: \n"); } - for (DrillbitEndpoint bit : newDrillbitSet) { + for (DrillbitEndpoint bit : UUIDtoEndpoints.values()) { builder.append('\t'); builder.append(bit.getAddress()); builder.append(':'); @@ -277,11 +271,13 @@ public DrillbitEndpoint apply( // Notify the drillbit listener for newly unregistered bits. if (!(unregisteredBits.isEmpty())) { - drillbitUnregistered(unregisteredBits); + drillbitUnregistered(unregisteredBits.entrySet().stream().collect( + Collectors.toMap(Map.Entry::getValue, Map.Entry::getKey))); } // Notify the drillbit listener for newly registered bits. if (!(registeredBits.isEmpty())) { - drillbitRegistered(registeredBits); + drillbitRegistered(registeredBits.entrySet().stream().collect( + Collectors.toMap(Map.Entry::getValue, Map.Entry::getKey))); } } catch (Exception e) { diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/zk/ZKRegistry.java b/drill-yarn/src/main/java/org/apache/drill/yarn/zk/ZKRegistry.java index d20c40eb414..3c04ef527e6 100644 --- a/drill-yarn/src/main/java/org/apache/drill/yarn/zk/ZKRegistry.java +++ b/drill-yarn/src/main/java/org/apache/drill/yarn/zk/ZKRegistry.java @@ -17,17 +17,11 @@ */ package org.apache.drill.yarn.zk; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import org.apache.drill.shaded.guava.com.google.common.annotations.VisibleForTesting; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint; import org.apache.drill.exec.work.foreman.DrillbitStatusListener; +import org.apache.drill.shaded.guava.com.google.common.annotations.VisibleForTesting; import org.apache.drill.yarn.appMaster.AMWrapperException; import org.apache.drill.yarn.appMaster.EventContext; import org.apache.drill.yarn.appMaster.Pollable; @@ -35,6 +29,12 @@ import org.apache.drill.yarn.appMaster.Task; import org.apache.drill.yarn.appMaster.TaskLifecycleListener; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + /** * AM-specific implementation of a Drillbit registry backed by ZooKeeper. * Listens to ZK events for registering a Drillbit and deregistering. Alerts the @@ -297,11 +297,12 @@ public AckEvent(Task task, DrillbitEndpoint endpoint) { * Callback from ZK to indicate that one or more drillbits have become * registered. We handle registrations in a critical section, then alert the * cluster controller outside the critical section. + * @param registeredDrillbitsUUID */ @Override - public void drillbitRegistered(Set registeredDrillbits) { - List updates = registerDrillbits(registeredDrillbits); + public void drillbitRegistered(Map registeredDrillbitsUUID) { + List updates = registerDrillbits(registeredDrillbitsUUID.keySet()); for (AckEvent event : updates) { if (event.task == null) { registryHandler.reserveHost(event.endpoint.getAddress()); @@ -363,12 +364,12 @@ private AckEvent drillbitRegistered(DrillbitEndpoint dbe) { * Callback from ZK to indicate that one or more drillbits have become * deregistered from ZK. We handle the deregistrations in a critical section, * but updates to the cluster controller outside of a critical section. + * @param unregisteredDrillbitsUUID */ @Override - public void drillbitUnregistered( - Set unregisteredDrillbits) { - List updates = unregisterDrillbits(unregisteredDrillbits); + public void drillbitUnregistered(Map unregisteredDrillbitsUUID) { + List updates = unregisterDrillbits(unregisteredDrillbitsUUID.keySet()); for (AckEvent event : updates) { registryHandler.completionAck(event.task, ENDPOINT_PROPERTY); } diff --git a/drill-yarn/src/test/java/org/apache/drill/yarn/zk/TestZkRegistry.java b/drill-yarn/src/test/java/org/apache/drill/yarn/zk/TestZkRegistry.java index 4263b01a00a..d2e88c9a112 100644 --- a/drill-yarn/src/test/java/org/apache/drill/yarn/zk/TestZkRegistry.java +++ b/drill-yarn/src/test/java/org/apache/drill/yarn/zk/TestZkRegistry.java @@ -17,15 +17,6 @@ */ package org.apache.drill.yarn.zk; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; - -import java.util.List; -import java.util.Map; -import java.util.Set; - import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; import org.apache.curator.retry.RetryNTimes; @@ -41,6 +32,15 @@ import org.apache.drill.yarn.zk.ZKRegistry.DrillbitTracker; import org.junit.Test; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + /** * Tests for the AM version of the cluster coordinator. The AM version has no * dependencies on the DoY config system or other systems, making it easy to @@ -108,14 +108,13 @@ private class TestDrillbitStatusListener implements DrillbitStatusListener { protected Set removed; @Override - public void drillbitUnregistered( - Set unregisteredDrillbits) { - removed = unregisteredDrillbits; + public void drillbitUnregistered(Map unregisteredDrillbitsUUID) { + removed = unregisteredDrillbitsUUID.keySet(); } @Override - public void drillbitRegistered(Set registeredDrillbits) { - added = registeredDrillbits; + public void drillbitRegistered(Map registeredDrillbitsUUID) { + added = registeredDrillbitsUUID.keySet(); } public void clear() { diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ClusterCoordinator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ClusterCoordinator.java index 75654171003..4ad79919215 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ClusterCoordinator.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ClusterCoordinator.java @@ -25,7 +25,6 @@ import java.util.Collection; import java.util.Map; -import java.util.Set; import java.util.concurrent.ConcurrentHashMap; /** @@ -107,17 +106,17 @@ public interface RegistrationHandle { /** * Actions to take when there are a set of new de-active drillbits. - * @param unregisteredBits + * @param unregisteredBitsUUID */ - protected void drillbitUnregistered(Set unregisteredBits) { + protected void drillbitUnregistered(Map unregisteredBitsUUID) { for (DrillbitStatusListener listener : listeners.keySet()) { - listener.drillbitUnregistered(unregisteredBits); + listener.drillbitUnregistered(unregisteredBitsUUID); } } - protected void drillbitRegistered(Set registeredBits) { + protected void drillbitRegistered(Map registeredBitsUUID) { for (DrillbitStatusListener listener : listeners.keySet()) { - listener.drillbitRegistered(registeredBits); + listener.drillbitRegistered(registeredBitsUUID); } } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/coord/zk/ZKClusterCoordinator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/coord/zk/ZKClusterCoordinator.java index 3d276c05683..7849a2ec53d 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/coord/zk/ZKClusterCoordinator.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/coord/zk/ZKClusterCoordinator.java @@ -31,6 +31,7 @@ import org.apache.curator.x.discovery.ServiceInstance; import org.apache.curator.x.discovery.details.ServiceCacheListener; import org.apache.drill.common.AutoCloseables; +import org.apache.drill.common.DrillNode; import org.apache.drill.common.config.DrillConfig; import org.apache.drill.exec.ExecConstants; import org.apache.drill.exec.coord.ClusterCoordinator; @@ -46,11 +47,8 @@ import java.io.IOException; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.Map; -import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; @@ -66,15 +64,13 @@ public class ZKClusterCoordinator extends ClusterCoordinator { private CuratorFramework curator; private ServiceDiscovery discovery; - private volatile Collection endpoints = Collections.emptyList(); private final String serviceName; private final CountDownLatch initialConnection = new CountDownLatch(1); private final TransientStoreFactory factory; private ServiceCache serviceCache; - private DrillbitEndpoint endpoint; // endpointsMap maps String UUID to Drillbit endpoints - private ConcurrentHashMap endpointsMap = new ConcurrentHashMap<>(); + private Map endpointsMap = new ConcurrentHashMap<>(); private static final Pattern ZK_COMPLEX_STRING = Pattern.compile("(^.*?)/(.*)/([^/]*)$"); public ZKClusterCoordinator(DrillConfig config, String connect) { @@ -220,7 +216,8 @@ public void unregister(RegistrationHandle handle) { */ public RegistrationHandle update(RegistrationHandle handle, State state) { ZKRegistrationHandle h = (ZKRegistrationHandle) handle; - try { + final DrillbitEndpoint endpoint; + try { endpoint = h.endpoint.toBuilder().setState(state).build(); ServiceInstance serviceInstance = ServiceInstance.builder() .name(serviceName) @@ -231,6 +228,7 @@ public RegistrationHandle update(RegistrationHandle handle, State state) { Throwables.throwIfUnchecked(e); throw new RuntimeException(e); } + handle.setEndPoint(endpoint); return handle; } @@ -282,46 +280,49 @@ public TransientStore getOrCreateTransientStore(final TransientStoreConfi private synchronized void updateEndpoints() { try { // All active bits in the Zookeeper - final Map activeEndpointsUUID = discovery.queryForInstances(serviceName).stream() + final Map UUIDtoEndpoints = discovery.queryForInstances(serviceName).stream() .collect(Collectors.toMap(ServiceInstance::getId, ServiceInstance::getPayload)); - final Map UUIDtoEndpoints = activeEndpointsUUID.entrySet().stream() - .collect(Collectors.toMap(Map.Entry::getValue, Map.Entry::getKey)); + final Map activeEndpointsUUID = UUIDtoEndpoints.entrySet().stream() + .collect(Collectors.toMap(x -> DrillNode.create(x.getValue()), Map.Entry::getKey)); // set of newly dead bits : original bits - new set of active bits. - Set unregisteredBits = new HashSet<>(); + final Map unregisteredBits = new HashMap<>(); // Set of newly live bits : new set of active bits - original bits. - Set registeredBits = new HashSet<>(); + final Map registeredBits = new HashMap<>(); // Updates the endpoints map if there is a change in state of the endpoint or with the addition // of new drillbit endpoints. Registered endpoints is set to newly live drillbit endpoints. - for (Map.Entry endpointToUUID : activeEndpointsUUID.entrySet()) { - endpointsMap.put(endpointToUUID.getKey(), endpointToUUID.getValue()); + for (Map.Entry endpoint : UUIDtoEndpoints.entrySet()) { + // check if this bit is newly added bit + if (!endpointsMap.containsKey(endpoint.getKey())) { + registeredBits.put(endpoint.getValue(), endpoint.getKey()); + } + endpointsMap.put(endpoint.getKey(), endpoint.getValue()); } // Remove all the endpoints that are newly dead for ( String bitUUID: endpointsMap.keySet()) { - if (!activeEndpointsUUID.containsKey(bitUUID)) { + if (!UUIDtoEndpoints.containsKey(bitUUID)) { final DrillbitEndpoint unregisteredBit = endpointsMap.get(bitUUID); - unregisteredBits.add(unregisteredBit); - - if (UUIDtoEndpoints.containsKey(unregisteredBit)) { + unregisteredBits.put(unregisteredBit, bitUUID); + final DrillNode unregisteredNode = DrillNode.create(unregisteredBit); + if (activeEndpointsUUID.containsKey(unregisteredNode)) { logger.info("Drillbit registered again with different UUID. [Details: Address: {}, UserPort: {}," + " PreviousUUID: {}, CurrentUUID: {}", unregisteredBit.getAddress(), unregisteredBit.getUserPort(), - bitUUID, UUIDtoEndpoints.get(unregisteredBit)); + bitUUID, activeEndpointsUUID.get(unregisteredNode)); } endpointsMap.remove(bitUUID); } } - endpoints = endpointsMap.values(); if (logger.isDebugEnabled()) { StringBuilder builder = new StringBuilder(); builder.append("Active drillbit set changed. Now includes "); - builder.append(activeEndpointsUUID.size()); + builder.append(UUIDtoEndpoints.size()); builder.append(" total bits. New active drillbits:\n"); builder.append("Address | User Port | Control Port | Data Port | Version | State\n"); - for (DrillbitEndpoint bit: activeEndpointsUUID.values()) { + for (DrillbitEndpoint bit: UUIDtoEndpoints.values()) { builder.append(bit.getAddress()).append(" | "); builder.append(bit.getUserPort()).append(" | "); builder.append(bit.getControlPort()).append(" | "); diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java index ea783225016..cf5a51fee59 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java @@ -56,6 +56,7 @@ import java.util.Collection; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; // TODO - consider re-name to PlanningContext, as the query execution context actually appears // in fragment contexts @@ -87,6 +88,9 @@ public enum SqlStatementType {OTHER, ANALYZE, CTAS, EXPLAIN, DESCRIBE_TABLE, DES */ private boolean closed = false; private DrillOperatorTable table; + private Map onlineEndpointsUUID; + private Map onlineEndpointNodesUUID; + private DrillNode localEndpointNode; public QueryContext(final UserSession session, final DrillbitContext drillbitContext, QueryId queryId) { this.drillbitContext = drillbitContext; @@ -225,6 +229,13 @@ public DrillbitEndpoint getCurrentEndpoint() { return drillbitContext.getEndpoint(); } + public DrillNode getCurrentEndpointNode() { + if (localEndpointNode == null) { + localEndpointNode = DrillNode.create(getCurrentEndpoint()); + } + return localEndpointNode; + } + public StoragePluginRegistry getStorage() { return drillbitContext.getStorage(); } @@ -242,11 +253,24 @@ public Collection getOnlineEndpoints() { } /** - * TODO: Change it to use {@link DrillNode} instead of DrillbitEndpoint * @return map of endpoint to UUIDs */ public Map getOnlineEndpointUUIDs() { - return drillbitContext.getOnlineEndpointUUIDs(); + if (onlineEndpointsUUID == null) { + onlineEndpointsUUID = drillbitContext.getOnlineEndpointUUIDs(); + } + return onlineEndpointsUUID; + } + + /** + * @return map of DrillNode to UUIDs + */ + public Map getOnlineEndpointNodeUUIDs() { + if (onlineEndpointNodesUUID == null) { + onlineEndpointNodesUUID = getOnlineEndpointUUIDs().entrySet().stream() + .collect(Collectors.toMap(x -> DrillNode.create(x.getKey()), Map.Entry::getValue)); + } + return onlineEndpointNodesUUID; } public DrillConfig getConfig() { diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/DistributedQueueParallelizer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/DistributedQueueParallelizer.java index c7c91f30b15..fecea5e7f6e 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/DistributedQueueParallelizer.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/DistributedQueueParallelizer.java @@ -23,18 +23,19 @@ import org.apache.drill.exec.ops.QueryContext; import org.apache.drill.exec.physical.PhysicalOperatorSetupException; import org.apache.drill.exec.physical.base.PhysicalOperator; +import org.apache.drill.common.DrillNode; import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint; import org.apache.drill.exec.resourcemgr.NodeResources; import org.apache.drill.exec.resourcemgr.config.QueryQueueConfig; import org.apache.drill.exec.resourcemgr.config.exception.QueueSelectionException; import org.apache.drill.exec.work.foreman.rm.QueryResourceManager; -import java.util.Map; -import java.util.HashMap; +import java.util.ArrayList; import java.util.Collection; -import java.util.Set; +import java.util.HashMap; import java.util.List; -import java.util.ArrayList; +import java.util.Map; +import java.util.Set; import java.util.function.BiFunction; import java.util.stream.Collectors; @@ -48,7 +49,7 @@ public class DistributedQueueParallelizer extends SimpleParallelizer { private final boolean planHasMemory; private final QueryContext queryContext; private final QueryResourceManager rm; - private final Map> operators; + private final Map> operators; public DistributedQueueParallelizer(boolean memoryPlanning, QueryContext queryContext, QueryResourceManager queryRM) { super(queryContext); @@ -62,7 +63,12 @@ public DistributedQueueParallelizer(boolean memoryPlanning, QueryContext queryCo public BiFunction getMemory() { return (endpoint, operator) -> { if (!planHasMemory) { - return operators.get(endpoint).get(operator); + final DrillNode drillEndpointNode = DrillNode.create(endpoint); + if (operator.isBufferedOperator(queryContext)) { + return operators.get(drillEndpointNode).get(operator); + } else { + return operator.getMaxAllocation(); + } } else { return operator.getMaxAllocation(); @@ -90,14 +96,18 @@ public void adjustMemory(PlanningSet planningSet, Set roots, if (planHasMemory) { return; } + + final Map onlineDrillNodeUUIDs = onlineEndpointUUIDs.entrySet().stream() + .collect(Collectors.toMap(x -> DrillNode.create(x.getKey()), x -> x.getValue())); + // total node resources for the query plan maintained per drillbit. - final Map totalNodeResources = - onlineEndpointUUIDs.keySet().stream().collect(Collectors.toMap(x ->x, + final Map totalNodeResources = + onlineDrillNodeUUIDs.keySet().stream().collect(Collectors.toMap(x -> x, x -> NodeResources.create())); // list of the physical operators and their memory requirements per drillbit. - final Map>> operators = - onlineEndpointUUIDs.keySet().stream().collect(Collectors.toMap(x -> x, + final Map>> operators = + onlineDrillNodeUUIDs.keySet().stream().collect(Collectors.toMap(x -> x, x -> new ArrayList<>())); for (Wrapper wrapper : roots) { @@ -112,14 +122,14 @@ public void adjustMemory(PlanningSet planningSet, Set roots, })); } - QueryQueueConfig queueConfig = null; + final QueryQueueConfig queueConfig; try { queueConfig = this.rm.selectQueue(max(totalNodeResources.values())); } catch (QueueSelectionException exception) { throw new ExecutionSetupException(exception.getMessage()); } - Map>> memoryAdjustedOperators = ensureOperatorMemoryWithinLimits(operators, totalNodeResources, queueConfig.getMaxQueryMemoryInMBPerNode()); memoryAdjustedOperators.entrySet().stream().forEach((x) -> { @@ -130,14 +140,14 @@ public void adjustMemory(PlanningSet planningSet, Set roots, this.operators.put(x.getKey(), memoryPerOperator); }); - this.rm.setCost(convertToUUID(totalNodeResources, onlineEndpointUUIDs)); + this.rm.setCost(convertToUUID(totalNodeResources, onlineDrillNodeUUIDs)); } - private Map convertToUUID(Map nodeResourcesMap, - Map onlineEndpointUUIDs) { + private Map convertToUUID(Map nodeResourcesMap, + Map onlineDrillNodeUUIDs) { Map nodeResourcesPerUUID = new HashMap<>(); - for (Map.Entry nodeResource : nodeResourcesMap.entrySet()) { - nodeResourcesPerUUID.put(onlineEndpointUUIDs.get(nodeResource.getKey()), nodeResource.getValue()); + for (Map.Entry nodeResource : nodeResourcesMap.entrySet()) { + nodeResourcesPerUUID.put(onlineDrillNodeUUIDs.get(nodeResource.getKey()), nodeResource.getValue()); } return nodeResourcesPerUUID; } @@ -160,11 +170,11 @@ private NodeResources max(Collection resources) { * @param nodeLimit permissible node limit. * @return list of operators which contain adjusted memory limits. */ - private Map>> - ensureOperatorMemoryWithinLimits(Map>> memoryPerOperator, - Map nodeResourceMap, long nodeLimit) { + private Map>> + ensureOperatorMemoryWithinLimits(Map>> memoryPerOperator, + Map nodeResourceMap, long nodeLimit) { // Get the physical operators which are above the node memory limit. - Map>> onlyMemoryAboveLimitOperators = new HashMap<>(); + Map>> onlyMemoryAboveLimitOperators = new HashMap<>(); memoryPerOperator.entrySet().stream().forEach((entry) -> { onlyMemoryAboveLimitOperators.putIfAbsent(entry.getKey(), new ArrayList<>()); if (nodeResourceMap.get(entry.getKey()).getMemoryInBytes() > nodeLimit) { @@ -175,7 +185,7 @@ private NodeResources max(Collection resources) { // Compute the total memory required by the physical operators on the drillbits which are above node limit. // Then use the total memory to adjust the memory requirement based on the permissible node limit. - Map>> memoryAdjustedDrillbits = new HashMap<>(); + Map>> memoryAdjustedDrillbits = new HashMap<>(); onlyMemoryAboveLimitOperators.entrySet().stream().forEach( entry -> { Long totalMemory = entry.getValue().stream().mapToLong(Pair::getValue).sum(); @@ -191,7 +201,7 @@ private NodeResources max(Collection resources) { // Get all the operations on drillbits which were adjusted for memory and merge them with operators which are not // adjusted for memory. - Map>> allDrillbits = new HashMap<>(); + Map>> allDrillbits = new HashMap<>(); memoryPerOperator.entrySet().stream().filter((entry) -> !memoryAdjustedDrillbits.containsKey(entry.getKey())).forEach( operatorMemory -> { allDrillbits.put(operatorMemory.getKey(), operatorMemory.getValue()); diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/MemoryCalculator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/MemoryCalculator.java index 94d319d4083..d3d759ca437 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/MemoryCalculator.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/MemoryCalculator.java @@ -23,15 +23,16 @@ import org.apache.drill.exec.physical.base.PhysicalOperator; import org.apache.drill.exec.physical.config.AbstractMuxExchange; import org.apache.drill.exec.planner.AbstractOpWrapperVisitor; +import org.apache.drill.common.DrillNode; import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint; import org.apache.drill.exec.resourcemgr.NodeResources; import org.apache.drill.shaded.guava.com.google.common.base.Preconditions; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Map.Entry; -import java.util.HashMap; import java.util.function.Function; import java.util.stream.Collectors; @@ -46,7 +47,7 @@ public class MemoryCalculator extends AbstractOpWrapperVisitor>> bufferedOperators; + private final Map>> bufferedOperators; private final QueryContext queryContext; public MemoryCalculator(PlanningSet planningSet, QueryContext context) { @@ -58,17 +59,17 @@ public MemoryCalculator(PlanningSet planningSet, QueryContext context) { // Helper method to compute the minor fragment count per drillbit. This method returns // a map with key as DrillbitEndpoint and value as the width (i.e #minorFragments) // per Drillbit. - private Map getMinorFragCountPerDrillbit(Wrapper currFragment) { + private Map getMinorFragCountPerDrillbit(Wrapper currFragment) { return currFragment.getAssignedEndpoints().stream() - .collect(Collectors.groupingBy(Function.identity(), + .collect(Collectors.groupingBy(x -> DrillNode.create(x), Collectors.summingInt(x -> 1))); } // Helper method to merge the memory computations for each operator given memory per operator // and the number of minor fragments per Drillbit. private void merge(Wrapper currFrag, - Map minorFragsPerDrillBit, - Function, Long> getMemory) { + Map minorFragsPerDrillBit, + Function, Long> getMemory) { NodeResources.merge(currFrag.getResourceMap(), minorFragsPerDrillBit.entrySet() @@ -92,15 +93,16 @@ public Void visitSendingExchange(Exchange exchange, Wrapper fragment) throws Run public Void visitReceivingExchange(Exchange exchange, Wrapper fragment) throws RuntimeException { final List receivingExchangePairs = fragment.getNode().getReceivingExchangePairs(); - final Map sendingFragsPerDrillBit = new HashMap<>(); + final Map sendingFragsPerDrillBit = new HashMap<>(); for(Fragment.ExchangeFragmentPair pair : receivingExchangePairs) { if (pair.getExchange() == exchange) { Wrapper sendingFragment = planningSet.get(pair.getNode()); Preconditions.checkArgument(sendingFragment.isEndpointsAssignmentDone()); for (DrillbitEndpoint endpoint : sendingFragment.getAssignedEndpoints()) { - sendingFragsPerDrillBit.putIfAbsent(endpoint, 0); - sendingFragsPerDrillBit.put(endpoint, sendingFragsPerDrillBit.get(endpoint)+1); + final DrillNode drillEndpointNode = DrillNode.create(endpoint); + sendingFragsPerDrillBit.putIfAbsent(drillEndpointNode, 0); + sendingFragsPerDrillBit.put(drillEndpointNode, sendingFragsPerDrillBit.get(drillEndpointNode)+1); } } } @@ -115,7 +117,7 @@ public Void visitReceivingExchange(Exchange exchange, Wrapper fragment) throws R return null; } - public List> getBufferedOperators(DrillbitEndpoint endpoint) { + public List> getBufferedOperators(DrillNode endpoint) { return this.bufferedOperators.getOrDefault(endpoint, new ArrayList<>()); } @@ -128,9 +130,9 @@ public Void visitOp(PhysicalOperator op, Wrapper fragment) { // minor fragments. Divide this memory estimation by fragment width to get the memory // requirement per minor fragment. long memoryCostPerMinorFrag = (int)Math.ceil(memoryCost/fragment.getAssignedEndpoints().size()); - Map drillbitEndpointMinorFragMap = getMinorFragCountPerDrillbit(fragment); + Map drillbitEndpointMinorFragMap = getMinorFragCountPerDrillbit(fragment); - Map> bufferedOperatorsPerDrillbit = drillbitEndpointMinorFragMap.entrySet().stream() .collect(Collectors.toMap((x) -> x.getKey(), diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Wrapper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Wrapper.java index fe8a46d791a..dfdeb6a71b5 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Wrapper.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Wrapper.java @@ -17,14 +17,6 @@ */ package org.apache.drill.exec.planner.fragment; -import java.util.List; -import java.util.Map; -import java.util.function.BinaryOperator; -import java.util.function.Function; -import java.util.stream.Collectors; - -import org.apache.drill.exec.resourcemgr.NodeResources; -import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList; import org.apache.drill.exec.physical.PhysicalOperatorSetupException; import org.apache.drill.exec.physical.base.AbstractPhysicalVisitor; import org.apache.drill.exec.physical.base.Exchange; @@ -33,11 +25,19 @@ import org.apache.drill.exec.physical.base.Store; import org.apache.drill.exec.physical.base.SubScan; import org.apache.drill.exec.planner.fragment.Fragment.ExchangeFragmentPair; +import org.apache.drill.common.DrillNode; import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint; - +import org.apache.drill.exec.resourcemgr.NodeResources; import org.apache.drill.shaded.guava.com.google.common.base.Preconditions; +import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList; import org.apache.drill.shaded.guava.com.google.common.collect.Lists; +import java.util.List; +import java.util.Map; +import java.util.function.BinaryOperator; +import java.util.function.Function; +import java.util.stream.Collectors; + /** * A wrapping class that allows us to add additional information to each fragment node for planning purposes. */ @@ -55,7 +55,7 @@ public class Wrapper { // A Drillbit can have n number of minor fragments then the NodeResource // contains cumulative resources required for all the minor fragments // for that major fragment on that Drillbit. - private Map nodeResourceMap; + private Map nodeResourceMap; // List of fragments this particular fragment depends on for determining its parallelization and endpoint assignments. private final List fragmentDependencies = Lists.newArrayList(); @@ -225,15 +225,15 @@ public void computeCpuResources() { return result; }; - Function cpuPerEndpoint = (endpoint) -> new NodeResources(1, 0); + Function cpuPerEndpoint = (endpoint) -> new NodeResources(1, 0); - nodeResourceMap = endpoints.stream() + nodeResourceMap = endpoints.stream().map(x -> DrillNode.create(x)) .collect(Collectors.groupingBy(Function.identity(), Collectors.reducing(NodeResources.create(), cpuPerEndpoint, merge))); } - public Map getResourceMap() { + public Map getResourceMap() { return nodeResourceMap; } } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/NodeResources.java b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/NodeResources.java index 76ab7f922df..5b92688f8ea 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/NodeResources.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/NodeResources.java @@ -25,7 +25,7 @@ import com.fasterxml.jackson.databind.DeserializationContext; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.deser.std.StdDeserializer; -import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint; +import org.apache.drill.common.DrillNode; import java.io.IOException; import java.util.Map; @@ -103,8 +103,8 @@ public void add(NodeResources other) { this.memoryInBytes += other.getMemoryInBytes(); } - public static Map merge(Map to, - Map from) { + public static Map merge(Map to, + Map from) { to.entrySet().stream().forEach((toEntry) -> toEntry.getValue().add(from.get(toEntry.getKey()))); return to; } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/config/QueryQueueConfigImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/config/QueryQueueConfigImpl.java index 6108f039306..85fedfcc4ed 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/config/QueryQueueConfigImpl.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/config/QueryQueueConfigImpl.java @@ -164,7 +164,7 @@ private NodeResources parseAndGetNodeShare(Config queueConfig) throws RMConfigEx @Override public String toString() { return "{ QueueName: " + queueName + ", QueueId: " + queueUUID + ", QueuePerNodeResource(MB): " + - queryPerNodeResourceShare.toString() + ", MaxQueryMemPerNode(MB): " + queryPerNodeResourceShare.toString() + + queueResourceShare.toString() + ", MaxQueryMemPerNode(MB): " + queryPerNodeResourceShare.toString() + ", MaxAdmissible: " + maxAdmissibleQuery + ", MaxWaiting: " + maxWaitingQuery + ", MaxWaitTimeout: " + maxWaitingTimeout + ", WaitForPreferredNodes: " + waitForPreferredNodes + "}"; } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/config/ResourcePool.java b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/config/ResourcePool.java index c2f8512242c..5f4e5577db3 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/config/ResourcePool.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/config/ResourcePool.java @@ -18,6 +18,7 @@ package org.apache.drill.exec.resourcemgr.config; import org.apache.drill.exec.ops.QueryContext; +import org.apache.drill.exec.resourcemgr.NodeResources; import org.apache.drill.exec.resourcemgr.config.selectors.ResourcePoolSelector; import java.util.List; @@ -67,4 +68,6 @@ public interface ResourcePool { List getChildPools(); ResourcePoolSelector getSelector(); + + NodeResources getPoolResourceShare(); } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/config/ResourcePoolImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/config/ResourcePoolImpl.java index 79cf6c82ff1..7842c02bcb3 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/config/ResourcePoolImpl.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/config/ResourcePoolImpl.java @@ -245,14 +245,19 @@ public ResourcePoolSelector getSelector() { return assignedSelector; } + @Override + public NodeResources getPoolResourceShare() { + return poolResourcePerNode; + } + @Override public String toString() { StringBuilder sb = new StringBuilder(); sb.append("{PoolName: ").append(poolName); sb.append(", PoolResourceShare: ").append(poolResourceShare); - sb.append(", Selector: ").append(assignedSelector.getSelectorType()); + sb.append(", Selector: ").append(assignedSelector.toString()); if (isLeafPool()) { - sb.append(", Queue: [").append(assignedQueue.toString()).append("]"); + sb.append(", Queue: ").append(assignedQueue.toString()); } else { sb.append(", ChildPools: ["); diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/config/ResourcePoolTree.java b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/config/ResourcePoolTree.java index 309bb24dbb2..b092c948426 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/config/ResourcePoolTree.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/config/ResourcePoolTree.java @@ -32,6 +32,8 @@ public interface ResourcePoolTree { ResourcePool getRootPool(); + NodeResources getRootPoolResources(); + Map getAllLeafQueues(); double getResourceShare(); diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/config/ResourcePoolTreeImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/config/ResourcePoolTreeImpl.java index 91b81f79606..c7c457e955e 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/config/ResourcePoolTreeImpl.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/config/ResourcePoolTreeImpl.java @@ -19,6 +19,7 @@ import com.typesafe.config.Config; import org.apache.drill.exec.ops.QueryContext; +import org.apache.drill.exec.proto.helper.QueryIdHelper; import org.apache.drill.exec.resourcemgr.NodeResources; import org.apache.drill.exec.resourcemgr.config.exception.QueueSelectionException; import org.apache.drill.exec.resourcemgr.config.exception.RMConfigException; @@ -93,6 +94,14 @@ public ResourcePool getRootPool() { return rootPool; } + /** + * @return NodeResources which represents total node resource + */ + @Override + public NodeResources getRootPoolResources() { + return rootPool.getPoolResourceShare(); + } + /** * @return Map containing all the configured leaf queues */ @@ -136,7 +145,7 @@ public QueryQueueConfig selectOneQueue(QueryContext queryContext, NodeResources final List selectedPools = assignmentResult.getSelectedLeafPools(); if (selectedPools.size() == 0) { throw new QueueSelectionException(String.format("No resource pools to choose from for the query: %s", - queryContext.getQueryId())); + QueryIdHelper.getQueryId(queryContext.getQueryId()))); } else if (selectedPools.size() == 1) { return selectedPools.get(0).getQueryQueue(); } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/config/selectionpolicy/BestFitQueueSelection.java b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/config/selectionpolicy/BestFitQueueSelection.java index a76f193363d..7de6d5e8513 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/config/selectionpolicy/BestFitQueueSelection.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/config/selectionpolicy/BestFitQueueSelection.java @@ -18,6 +18,7 @@ package org.apache.drill.exec.resourcemgr.config.selectionpolicy; import org.apache.drill.exec.ops.QueryContext; +import org.apache.drill.exec.proto.helper.QueryIdHelper; import org.apache.drill.exec.resourcemgr.NodeResources; import org.apache.drill.exec.resourcemgr.config.QueryQueueConfig; import org.apache.drill.exec.resourcemgr.config.ResourcePool; @@ -70,9 +71,10 @@ public int compare(ResourcePool o1, ResourcePool o2) { @Override public ResourcePool selectQueue(List allPools, QueryContext queryContext, NodeResources maxResourcePerNode) throws QueueSelectionException { + final String queryIdString = QueryIdHelper.getQueryId(queryContext.getQueryId()); if (allPools.isEmpty()) { throw new QueueSelectionException(String.format("There are no pools to apply %s selection policy pool for the " + - "query: %s", getSelectionPolicy().toString(), queryContext.getQueryId())); + "query: %s", getSelectionPolicy().toString(), queryIdString)); } allPools.sort(new BestFitComparator()); @@ -85,9 +87,8 @@ public ResourcePool selectQueue(List allPools, QueryContext queryC break; } } - logger.debug("Selected pool {} based on {} policy for query {}", selectedPool.getPoolName(), - getSelectionPolicy().toString(), - queryContext.getQueryId()); + logger.info("Selected pool {} based on {} policy for query {}", selectedPool.getPoolName(), + getSelectionPolicy().toString(), queryIdString); return selectedPool; } } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/config/selectionpolicy/DefaultQueueSelection.java b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/config/selectionpolicy/DefaultQueueSelection.java index f1c03c3f501..e43697c2996 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/config/selectionpolicy/DefaultQueueSelection.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/config/selectionpolicy/DefaultQueueSelection.java @@ -18,6 +18,7 @@ package org.apache.drill.exec.resourcemgr.config.selectionpolicy; import org.apache.drill.exec.ops.QueryContext; +import org.apache.drill.exec.proto.helper.QueryIdHelper; import org.apache.drill.exec.resourcemgr.NodeResources; import org.apache.drill.exec.resourcemgr.config.ResourcePool; import org.apache.drill.exec.resourcemgr.config.exception.QueueSelectionException; @@ -39,14 +40,15 @@ public DefaultQueueSelection() { @Override public ResourcePool selectQueue(List allPools, QueryContext queryContext, NodeResources maxResourcePerNode) throws QueueSelectionException { + final String queryIdString = QueryIdHelper.getQueryId(queryContext.getQueryId()); for (ResourcePool pool : allPools) { if (pool.isDefaultPool()) { - logger.debug("Selected default pool: {} for the query: {}", pool.getPoolName(), queryContext.getQueryId()); + logger.debug("Selected default pool: {} for the query: {}", pool.getPoolName(), queryIdString); return pool; } } throw new QueueSelectionException(String.format("There is no default pool to select from list of pools provided " + - "for the query: %s", queryContext.getQueryId())); + "for the query: %s", queryIdString)); } } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/config/selectionpolicy/RandomQueueSelection.java b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/config/selectionpolicy/RandomQueueSelection.java index 63c51f29d45..6eb29e08480 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/config/selectionpolicy/RandomQueueSelection.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/config/selectionpolicy/RandomQueueSelection.java @@ -18,6 +18,7 @@ package org.apache.drill.exec.resourcemgr.config.selectionpolicy; import org.apache.drill.exec.ops.QueryContext; +import org.apache.drill.exec.proto.helper.QueryIdHelper; import org.apache.drill.exec.resourcemgr.NodeResources; import org.apache.drill.exec.resourcemgr.config.ResourcePool; import org.apache.drill.exec.resourcemgr.config.exception.QueueSelectionException; @@ -39,13 +40,14 @@ public RandomQueueSelection() { @Override public ResourcePool selectQueue(List allPools, QueryContext queryContext, NodeResources maxResourcePerNode) throws QueueSelectionException { + final String queryIdString = QueryIdHelper.getQueryId(queryContext.getQueryId()); if (allPools.size() == 0) { throw new QueueSelectionException(String.format("Input pool list is empty to apply %s selection policy", getSelectionPolicy().toString())); } Collections.shuffle(allPools); ResourcePool selectedPool = allPools.get(0); - logger.debug("Selected random pool: {} for query: {}", selectedPool.getPoolName(), queryContext.getQueryId()); + logger.debug("Selected random pool: {} for query: {}", selectedPool.getPoolName(), queryIdString); return selectedPool; } } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/RMBlobStoreManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/RMBlobStoreManager.java index 2cfa654e851..713d74489d0 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/RMBlobStoreManager.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/RMBlobStoreManager.java @@ -32,8 +32,12 @@ */ public interface RMBlobStoreManager { void reserveResources(Map queryResourceAssignment, QueryQueueConfig selectedQueue, - String leaderId, String queryId, String foremanNode) throws Exception; + String leaderId, String queryId, String foremanUUID) throws Exception; - void freeResources(Map queryResourceAssignment, QueryQueueConfig selectedQueue, - String leaderId, String queryId, String foremanNode) throws Exception; + String freeResources(Map queryResourceAssignment, QueryQueueConfig selectedQueue, + String leaderId, String queryId, String foremanUUID) throws Exception; + + void registerResource(String selfUUID, NodeResources resourceToRegister) throws Exception; + + void updateLeadershipInformation(String queueName, String leaderUUID) throws Exception; } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/RMConsistentBlobStoreManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/RMConsistentBlobStoreManager.java index 33e719df9fb..befa4bce838 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/RMConsistentBlobStoreManager.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/RMConsistentBlobStoreManager.java @@ -30,6 +30,7 @@ import org.apache.drill.exec.resourcemgr.NodeResources; import org.apache.drill.exec.resourcemgr.NodeResources.NodeResourcesDe; import org.apache.drill.exec.resourcemgr.config.QueryQueueConfig; +import org.apache.drill.exec.resourcemgr.config.ResourcePoolTree; import org.apache.drill.exec.resourcemgr.rmblobmgr.exception.LeaderChangeException; import org.apache.drill.exec.resourcemgr.rmblobmgr.exception.RMBlobUpdateException; import org.apache.drill.exec.resourcemgr.rmblobmgr.exception.ResourceUnavailableException; @@ -81,12 +82,15 @@ public class RMConsistentBlobStoreManager implements RMBlobStoreManager { private final Map rmStateBlobs; + private final ResourcePoolTree resourceTree; + private final StringBuilder exceptionStringBuilder = new StringBuilder(); - public RMConsistentBlobStoreManager(DrillbitContext context, Collection leafQueues) throws + public RMConsistentBlobStoreManager(DrillbitContext context, ResourcePoolTree poolTree) throws StoreException { try { this.context = context; + this.resourceTree = poolTree; this.serDeMapper = initializeMapper(context.getClasspathScan()); this.rmBlobStore = (ZookeeperTransactionalPersistenceStore) context.getStoreProvider() .getOrCreateStore(PersistentStoreConfig.newJacksonBuilder(serDeMapper, RMStateBlob.class) @@ -96,7 +100,7 @@ public RMConsistentBlobStoreManager(DrillbitContext context, Collection(); - initializeBlobs(leafQueues); + initializeBlobs(resourceTree.getAllLeafQueues().values()); } catch (StoreException ex) { throw ex; } catch (Exception ex) { @@ -148,8 +152,8 @@ private void initializeBlobs(Collection leafQueues) throws Exc // if here that means lock is acquired rmStateBlobs.put(ClusterStateBlob.NAME, new ClusterStateBlob(RM_STATE_BLOB_VERSION, new HashMap<>())); - rmStateBlobs.put(QueueLeadershipBlob.NAME, - new QueueLeadershipBlob(RM_STATE_BLOB_VERSION, new HashMap<>())); + final Map queueLeaders = new HashMap<>(); + rmStateBlobs.put(QueueLeadershipBlob.NAME, new QueueLeadershipBlob(RM_STATE_BLOB_VERSION, queueLeaders)); // This ForemanResourceUsage blob needs to be per queue final ForemanQueueUsageBlob queueUsageBlob = new ForemanQueueUsageBlob(RM_STATE_BLOB_VERSION, new HashMap<>()); @@ -177,30 +181,106 @@ private void initializeBlobs(Collection leafQueues) throws Exc @Override public void reserveResources(Map queryResourceAssignment, QueryQueueConfig selectedQueue, String leaderId, - String queryId, String foremanNode) throws Exception { - // Looks like leader hasn't changed yet so let's try to reserve the resources - // See if the call is to reserve or free up resources - Map resourcesMap = queryResourceAssignment; - resourcesMap = queryResourceAssignment.entrySet().stream() + String queryId, String foremanUUID) throws Exception { + acquireLockAndUpdate(queryResourceAssignment, selectedQueue, leaderId, queryId, foremanUUID, false); + } + + @Override + public String freeResources(Map queryResourceAssignment, + QueryQueueConfig selectedQueue, String leaderId, + String queryId, String foremanUUID) throws Exception { + final Map resourcesMap = queryResourceAssignment.entrySet().stream() .collect(Collectors.toMap(Map.Entry::getKey, (x) -> new NodeResources(x.getValue().getVersion(), - -x.getValue().getMemoryInBytes(), - -x.getValue().getNumVirtualCpu()))); - acquireLockAndUpdate(resourcesMap, selectedQueue, leaderId, queryId, foremanNode); + -x.getValue().getMemoryInBytes(), + -x.getValue().getNumVirtualCpu()))); + return acquireLockAndUpdate(resourcesMap, selectedQueue, leaderId, queryId, foremanUUID, true); } @Override - public void freeResources(Map queryResourceAssignment, - QueryQueueConfig selectedQueue, String leaderId, - String queryId, String foremanNode) throws Exception { - acquireLockAndUpdate(queryResourceAssignment, selectedQueue, leaderId, queryId, foremanNode); + public void registerResource(String selfUUID, NodeResources resourceToRegister) throws Exception { + try { + globalBlobMutex.acquire(); + } catch (Exception ex) { + logger.error("Failed on acquiring the global mutex while registering self resources in blob"); + throw ex; + } + + try { + // get the current cluster state blob + final ClusterStateBlob clusterBlob = (ClusterStateBlob) rmBlobStore.get(ClusterStateBlob.NAME, null); + final Map currentClusterState = clusterBlob.getClusterState(); + if (currentClusterState.containsKey(selfUUID)) { + throw new RMBlobUpdateException(String.format("Drillbit with UUID %s is already present in the cluster state " + + "blob. This can only happen if 2 Drillbits are assigned same unique id", selfUUID)); + } + + currentClusterState.put(selfUUID, resourceToRegister); + clusterBlob.setClusterState(currentClusterState); + + // write the updated cluster state blob + rmBlobStore.put(ClusterStateBlob.NAME, clusterBlob); + } catch (Exception ex) { + logger.error("Failed to register resource in the blob", ex); + throw new RMBlobUpdateException("Failed to register resource in the blob", ex); + } finally { + // Check if the caller has acquired the mutex + if (globalBlobMutex.isAcquiredInThisProcess()) { + try { + globalBlobMutex.release(); + } catch (Exception ex) { + logger.error("Failed on releasing the global mutex while registering self resources in blob", ex); + // don't throw this release exception instead throw the original exception if any. Since release exception + // should not matter much + } + } + } } - private void updateBlobs(Map resourcesMap, QueryQueueConfig selectedQueue, - String leaderId, String queryId, String foremanNode) throws Exception { + @Override + public void updateLeadershipInformation(String queueName, String leaderUUID) throws Exception { + try { + globalBlobMutex.acquire(); + } catch (Exception ex) { + logger.error("Failed on acquiring the global mutex while updating queue leader in blob"); + throw ex; + } + + try { + // get the current cluster state blob + final QueueLeadershipBlob queueLeaderBlob = (QueueLeadershipBlob) rmBlobStore.get(QueueLeadershipBlob.NAME, + null); + final Map currentQueueLeaders = queueLeaderBlob.getQueueLeaders(); + final String oldLeaderId = currentQueueLeaders.put(queueName, leaderUUID); + + logger.info("Updating the leadership information for queue. [Details: QueueName: {}, OldLeader: {}, NewLeader: " + + "{}]", queueName, oldLeaderId == null ? "" : oldLeaderId, leaderUUID); + + // write the updated cluster state blob + rmBlobStore.put(QueueLeadershipBlob.NAME, queueLeaderBlob); + } catch (Exception ex) { + logger.error("Failed to update queue leadership information in the blob", ex); + throw new RMBlobUpdateException("Failed to update queue leadership information in the blob", ex); + } finally { + // Check if the caller has acquired the mutex + if (globalBlobMutex.isAcquiredInThisProcess()) { + try { + globalBlobMutex.release(); + } catch (Exception ex) { + logger.error("Failed on releasing the global mutex while updating queue leader in blob", ex); + // don't throw this release exception instead throw the original exception if any. Since release exception + // should not matter much + } + } + } + } + + private String updateBlobs(Map resourcesMap, QueryQueueConfig selectedQueue, + String leaderId, String queryId, String foremanUUID, boolean freeResources) + throws Exception { exceptionStringBuilder.append("QueryId: ").append(queryId) - .append(", ForemanBit: ").append(foremanNode) + .append(", ForemanBit: ").append(foremanUUID) .append(", QueueName: ").append(selectedQueue.getQueueName()) .append(", Admitted Leader: ").append(leaderId); @@ -217,17 +297,38 @@ private void updateBlobs(Map resourcesMap, QueryQueueConf throw new RMBlobUpdateException(String.format("Failed to get one or more blob while update. [Details: %s]", exceptionStringBuilder.toString())); } else { + // remove the previous blobs information and store the recent one which we read from zookeeper + rmStateBlobs.clear(); rmStateBlobs.putAll(rmBlobs); } // Check if the leader admitting the query is still leader of the queue - final String currentQueueLeader = ((QueueLeadershipBlob)rmStateBlobs.get(QueueLeadershipBlob.NAME)) + String currentQueueLeader = ((QueueLeadershipBlob)rmStateBlobs.get(QueueLeadershipBlob.NAME)) .getQueueLeaders().get(selectedQueue.getQueueName()); - if (currentQueueLeader == null || !currentQueueLeader.equals(leaderId)) { - throw new LeaderChangeException(String.format("The leader which admitted the query in queue doesn't match " + - "current leader %s of the queue [Details: %s]", currentQueueLeader, exceptionStringBuilder.toString())); + + String logString; + // usually can happen if the queueLeaderShip blob is not initialized + if (currentQueueLeader == null) { + logString = String.format("There is no leader information about the queue which admitted the query. " + + "[Details: %s]", exceptionStringBuilder.toString()); + if (freeResources) { + // should be here while freeing up resource so it's fine to use old leader while still updating the blobs + logger.info(logString); + logger.info("Using the old leader {}", leaderId); + currentQueueLeader = leaderId; + } else { + throw new LeaderChangeException(logString); + } + } else if (!currentQueueLeader.equals(leaderId)) { + logString = String.format("The leader which admitted the query in queue doesn't match current leader %s of the " + + "queue [Details: %s]", currentQueueLeader, exceptionStringBuilder.toString()); + if (freeResources) { + logger.info(logString); + } else { + throw new LeaderChangeException(logString); + } } - // Remove leadership blob from cache + // Remove leadership blob from cache since we don't have to update this blob rmStateBlobs.remove(QueueLeadershipBlob.NAME); // Cluster state blob @@ -237,7 +338,12 @@ private void updateBlobs(Map resourcesMap, QueryQueueConf // ForemanResourceUsage blob final ForemanQueueUsageBlob resourceUsageBlob = (ForemanQueueUsageBlob)rmStateBlobs.get(queueBlobName); final Map allForemanUsage = resourceUsageBlob.getAllForemanInfo(); - final ForemanResourceUsage currentUsage = allForemanUsage.get(foremanNode); + ForemanResourceUsage currentUsage = allForemanUsage.get(foremanUUID); + + if (currentUsage == null) { + // there is no usage registered by this foreman bit yet on this queue so create a default instance + currentUsage = new ForemanResourceUsage(RM_STATE_BLOB_VERSION, new HashMap<>(), 0); + } final Map usageMapAcrossDrillbits = currentUsage.getForemanUsage(); int currentRunningCount = currentUsage.getRunningCount(); @@ -247,14 +353,20 @@ private void updateBlobs(Map resourcesMap, QueryQueueConf final long memoryToReserve = bitResourcesToReserve.getMemoryInBytes(); if (!currentClusterState.containsKey(bitUUID)) { - throw new RMBlobUpdateException(String.format("Drillbit with UUID %s which is assigned to query is " + - "not found in ClusterState blob. [Details: %s]", bitUUID, exceptionStringBuilder.toString())); + logString = String.format("Drillbit with UUID %s which is assigned to query is " + + "not found in ClusterState blob. [Details: %s]", bitUUID, exceptionStringBuilder.toString()); + if (freeResources) { + logger.info(logString); + continue; + } else { + throw new RMBlobUpdateException(logString); + } } final NodeResources bitAvailableResources = currentClusterState.get(bitUUID); long currentAvailableMemory = bitAvailableResources.getMemoryInBytes(); if (currentAvailableMemory < memoryToReserve) { throw new ResourceUnavailableException(String.format("Drillbit with UUID %s which is assigned to query " + - "doesn't have enough memory available. [Details: %s, AvailableMemory: %s, RequiredMemory: %s]", bitUUID, + "doesn't have enough memory available. [Details: AvailableMemory: %s, RequiredMemory: %s, %s]", bitUUID, currentAvailableMemory, memoryToReserve, exceptionStringBuilder.toString())); } // Update local ClusterState @@ -262,9 +374,12 @@ private void updateBlobs(Map resourcesMap, QueryQueueConf currentClusterState.put(bitUUID, bitAvailableResources); // Update local ForemanResourceUsage for foremanNode with this query resource ask - final NodeResources currentState = usageMapAcrossDrillbits.get(bitUUID); + NodeResources currentState = usageMapAcrossDrillbits.get(bitUUID); + if (currentState == null) { + currentState = new NodeResources(0, 0); + } long availableMemory = currentState.getMemoryInBytes(); - currentState.setMemoryInBytes(availableMemory - memoryToReserve); + currentState.setMemoryInBytes(availableMemory + memoryToReserve); usageMapAcrossDrillbits.put(bitUUID, currentState); } @@ -272,9 +387,10 @@ private void updateBlobs(Map resourcesMap, QueryQueueConf currentClusterBlob.setClusterState(currentClusterState); // update the local ForemanQueueUsageBlob with final ForemanResourceUsage - currentUsage.setRunningCount(currentRunningCount + 1); + final int updatedRunningCount = currentRunningCount + ((freeResources) ? -1 : 1); + currentUsage.setRunningCount(updatedRunningCount); currentUsage.setForemanUsage(usageMapAcrossDrillbits); - allForemanUsage.put(foremanNode, currentUsage); + allForemanUsage.put(foremanUUID, currentUsage); resourceUsageBlob.setAllForemanInfo(allForemanUsage); // Update local blob cache @@ -292,10 +408,14 @@ private void updateBlobs(Map resourcesMap, QueryQueueConf // Reset the exceptionStringBuilder for next event exceptionStringBuilder.delete(0, exceptionStringBuilder.length()); + + return currentQueueLeader; } - private void acquireLockAndUpdate(Map queryResourceAssignment, QueryQueueConfig selectedQueue, - String leaderId, String queryId, String foremanNode) throws Exception { + private String acquireLockAndUpdate(Map queryResourceAssignment, + QueryQueueConfig selectedQueue, String leaderId, + String queryId, String foremanUUID, boolean freeResources) + throws Exception { try { globalBlobMutex.acquire(); } catch (Exception ex) { @@ -304,7 +424,7 @@ private void acquireLockAndUpdate(Map queryResourceAssign } try { - updateBlobs(queryResourceAssignment, selectedQueue, leaderId, queryId, foremanNode); + return updateBlobs(queryResourceAssignment, selectedQueue, leaderId, queryId, foremanUUID, freeResources); } catch (Exception ex) { logger.error("Failed to update the blobs", ex); throw ex; @@ -333,6 +453,11 @@ public boolean writeAllRMBlobs(Map rmStateBlobs) { return rmBlobStore.putAsTransaction(rmStateBlobs); } + @VisibleForTesting + public boolean deleteAllRMBlobs(List rmStateBlobs) { + return rmBlobStore.deleteAsTransaction(rmStateBlobs); + } + @VisibleForTesting public Map serializePassedInBlob(Map inputBlobs) throws Exception { Map serializedBlobs = new HashMap<>(); diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/rmblob/ClusterStateBlob.java b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/rmblob/ClusterStateBlob.java index 7faefcf4bf4..73e7a2e2f75 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/rmblob/ClusterStateBlob.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/rmblob/ClusterStateBlob.java @@ -28,6 +28,7 @@ public class ClusterStateBlob extends AbstractRMStateBlob { public static final String NAME = "cluster_usage"; + // Stores the resources available out of total resources on each node of the cluster @JsonDeserialize(contentUsing = NodeResources.NodeResourcesDe.class) private Map clusterState; diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/rmblob/ForemanResourceUsage.java b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/rmblob/ForemanResourceUsage.java index de6f1fed80e..936798ecb02 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/rmblob/ForemanResourceUsage.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/rmblob/ForemanResourceUsage.java @@ -33,6 +33,7 @@ public class ForemanResourceUsage { private int version; + // Stores the resources used across all queries of a Foreman on all the other Drillbit nodes private Map foremanUsage; private int runningCount; diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserSession.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserSession.java index 0798deabb85..7d404692598 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserSession.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserSession.java @@ -17,23 +17,13 @@ */ package org.apache.drill.exec.rpc.user; -import java.io.IOException; -import java.util.List; -import java.util.Map; -import java.util.UUID; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.atomic.AtomicInteger; - -import org.apache.drill.exec.ExecConstants; -import org.apache.drill.shaded.guava.com.google.common.base.Preconditions; -import org.apache.drill.shaded.guava.com.google.common.base.Strings; - import org.apache.calcite.schema.Schema; import org.apache.calcite.schema.SchemaPlus; import org.apache.calcite.schema.Table; import org.apache.calcite.tools.ValidationException; import org.apache.drill.common.config.DrillConfig; import org.apache.drill.common.config.DrillProperties; +import org.apache.drill.exec.ExecConstants; import org.apache.drill.exec.planner.physical.PlannerSettings; import org.apache.drill.exec.planner.sql.SchemaUtilites; import org.apache.drill.exec.planner.sql.handlers.SqlHandlerUtil; @@ -41,15 +31,23 @@ import org.apache.drill.exec.proto.UserProtos.UserProperties; import org.apache.drill.exec.server.options.OptionManager; import org.apache.drill.exec.server.options.SessionOptionManager; - -import org.apache.drill.shaded.guava.com.google.common.collect.Maps; import org.apache.drill.exec.store.AbstractSchema; import org.apache.drill.exec.store.StorageStrategy; import org.apache.drill.exec.store.dfs.DrillFileSystem; import org.apache.drill.exec.store.dfs.WorkspaceSchemaFactory; +import org.apache.drill.shaded.guava.com.google.common.base.Preconditions; +import org.apache.drill.shaded.guava.com.google.common.base.Strings; +import org.apache.drill.shaded.guava.com.google.common.collect.Maps; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicInteger; + public class UserSession implements AutoCloseable { private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UserSession.class); @@ -126,7 +124,7 @@ private boolean canApplyUserProperty() { sb.append(DrillProperties.QUOTING_IDENTIFIERS).append(","); } - if (userSession.properties.containsKey(DrillProperties.QUERY_TAGS)) { + if (userSession.properties.containsKey(DrillProperties.QUERY_TAGS.toLowerCase())) { sb.append(DrillProperties.QUERY_TAGS); } @@ -145,9 +143,9 @@ public UserSession build() { userSession.properties.getProperty(DrillProperties.QUOTING_IDENTIFIERS)); } - if (userSession.properties.containsKey(DrillProperties.QUERY_TAGS)) { + if (userSession.properties.containsKey(DrillProperties.QUERY_TAGS.toLowerCase())) { userSession.setSessionOption(ExecConstants.RM_QUERY_TAGS_KEY, - userSession.properties.getProperty(DrillProperties.QUERY_TAGS)); + userSession.properties.getProperty(DrillProperties.QUERY_TAGS.toLowerCase())); } } UserSession session = userSession; diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java index 215fbb0d830..5d0de865308 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java @@ -228,9 +228,12 @@ public void run() throws Exception { int httpPort = getWebServerPort(); md = md.toBuilder().setHttpPort(httpPort).build(); } - registrationHandle = coord.register(md); + // Must start the RM after the above since it needs to read system options. drillbitContext.startRM(); + registrationHandle = coord.register(md); + // Set the registration handle for local bit + drillbitContext.setRegistrationHandle(registrationHandle); shutdownHook = new ShutdownThread(this, new StackTrace()); Runtime.getRuntime().addShutdownHook(shutdownHook); diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java index bbcf0e42465..c3259f80f29 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java @@ -24,6 +24,7 @@ import org.apache.drill.common.scanner.persistence.ScanResult; import org.apache.drill.exec.compile.CodeCompiler; import org.apache.drill.exec.coord.ClusterCoordinator; +import org.apache.drill.exec.coord.ClusterCoordinator.RegistrationHandle; import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry; import org.apache.drill.exec.expr.fn.registry.RemoteFunctionRegistry; import org.apache.drill.exec.memory.BufferAllocator; @@ -50,11 +51,14 @@ import java.util.Map.Entry; import java.util.Set; import java.util.concurrent.ExecutorService; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import static org.apache.drill.shaded.guava.com.google.common.base.Preconditions.checkNotNull; public class DrillbitContext implements AutoCloseable { -// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillbitContext.class); + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillbitContext.class); private final BootStrapContext context; private final PhysicalPlanReader reader; @@ -75,6 +79,10 @@ public class DrillbitContext implements AutoCloseable { private final DrillOperatorTable table; private final QueryProfileStoreContext profileStoreContext; private ResourceManager resourceManager; + private RegistrationHandle handle; + private final Lock isHandleSetLock = new ReentrantLock(); + private final Condition isHandleSetCondition = isHandleSetLock.newCondition(); + public DrillbitContext( DrillbitEndpoint endpoint, @@ -139,6 +147,30 @@ public void startRM() { resourceManager = new ResourceManagerBuilder(this).build(); } + public void setRegistrationHandle(RegistrationHandle handle) { + try { + isHandleSetLock.lock(); + this.handle = handle; + } finally { + isHandleSetCondition.signal(); + isHandleSetLock.unlock(); + } + } + + public RegistrationHandle getRegistrationHandle() { + isHandleSetLock.lock(); + while (handle == null) { + try { + isHandleSetCondition.await(); + } catch (InterruptedException ex) { + logger.debug("Interrupted while waiting to get registration handle"); + // continue + } + } + isHandleSetLock.unlock(); + return handle; + } + public FunctionImplementationRegistry getFunctionImplementationRegistry() { return functionRegistry; } @@ -301,6 +333,7 @@ public void close() throws Exception { getFunctionImplementationRegistry().close(); getRemoteFunctionRegistry().close(); getCompiler().close(); + getResourceManager().close(); } public ResourceManager getResourceManager() { diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/DrillbitStatusListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/DrillbitStatusListener.java index e0f6bc1b91b..175f151ee1c 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/DrillbitStatusListener.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/DrillbitStatusListener.java @@ -18,9 +18,9 @@ package org.apache.drill.exec.work.foreman; -import org.apache.drill.exec.proto.CoordinationProtos; +import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint; -import java.util.Set; +import java.util.Map; /** * Interface to define the listener to take actions when the set of active drillbits is changed. @@ -29,14 +29,14 @@ public interface DrillbitStatusListener { /** * The action to taken when a set of drillbits are unregistered from the cluster. - * @param unregisteredDrillbits the set of newly unregistered drillbits. + * @param unregisteredDrillbitsUUID */ - void drillbitUnregistered(Set unregisteredDrillbits); + void drillbitUnregistered(Map unregisteredDrillbitsUUID); /** * The action to taken when a set of new drillbits are registered to the cluster. - * @param registeredDrillbits the set of newly registered drillbits. Note: the complete set of currently registered bits could be different. + * @param registeredDrillbitsUUID */ - void drillbitRegistered(Set registeredDrillbits); + void drillbitRegistered(Map registeredDrillbitsUUID); } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java index 62e6846b8be..aad01c0eb16 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java @@ -61,6 +61,7 @@ import org.apache.drill.exec.work.filter.RuntimeFilterRouter; import org.apache.drill.exec.work.foreman.rm.QueryQueue.QueueTimeoutException; import org.apache.drill.exec.work.foreman.rm.QueryQueue.QueryQueueException; +import org.apache.drill.exec.work.foreman.rm.QueryResourceManager.QueryAdmitResponse; import org.apache.drill.exec.work.foreman.rm.QueryResourceManager; import org.apache.drill.shaded.guava.com.google.common.base.Preconditions; import org.apache.drill.shaded.guava.com.google.common.collect.Lists; @@ -432,7 +433,6 @@ private void runPhysicalPlan(final PhysicalPlan plan, Pointer textPlan) if (textPlan != null) { queryManager.setPlanText(textPlan.value); } - queryRM.setCost(plan.totalCost()); queryManager.setTotalCost(plan.totalCost()); work.applyPlan(drillbitContext.getPlanReader()); logWorkUnit(work); @@ -508,12 +508,10 @@ private void startAdmittedQuery() { private void reserveAndRunFragments() { // Now try to reserve the resources required by this query try { - // TODO: pass parameters for reserveResources - if (!queryRM.reserveResources(null, queryId)) { + if (!queryRM.reserveResources()) { // query is added to RM waiting queue - // TODO: Add the queue name logger.info("Query {} is added to the RM waiting queue of rm pool {} since it was not able to reserve " + - "required resources", queryId); + "required resources", queryIdString, queryRM.queueName()); return; } runFragments(); @@ -530,10 +528,7 @@ private void reserveAndRunFragments() { private boolean enqueue() { queryStateProcessor.moveToState(QueryState.ENQUEUED, null); try { - if (queryRM.admit() == QueryResourceManager.QueryAdmitResponse.WAIT_FOR_RESPONSE) { - return false; - } - return true; + return queryRM.admit() != QueryAdmitResponse.WAIT_FOR_RESPONSE; } catch (QueueTimeoutException | QueryQueueException e) { queryStateProcessor.moveToState(QueryState.FAILED, e); return false; @@ -593,7 +588,7 @@ private void runPreparedStatement(final PreparedStatementHandle preparedStatemen } queryText = serverState.getSqlQuery(); - logger.info("Prepared statement query for QueryId {} : {}", queryId, queryText); + logger.info("Prepared statement query for QueryId {} : {}", queryIdString, queryText); runSQL(queryText); } @@ -625,8 +620,7 @@ private void logWorkUnit(QueryWorkUnit queryWorkUnit) { if (! logger.isTraceEnabled()) { return; } - logger.trace(String.format("PlanFragments for query %s \n%s", - queryId, queryWorkUnit.stringifyFragments())); + logger.trace(String.format("PlanFragments for query %s \n%s", queryIdString, queryWorkUnit.stringifyFragments())); } private void runSQL(final String sql) throws ExecutionSetupException { diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java index 0c140a45c8f..730c207e7eb 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java @@ -17,14 +17,10 @@ */ package org.apache.drill.exec.work.foreman; +import com.carrotsearch.hppc.IntObjectHashMap; +import com.carrotsearch.hppc.predicates.IntObjectPredicate; import com.fasterxml.jackson.core.JsonProcessingException; import io.netty.buffer.ByteBuf; - -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.atomic.AtomicInteger; - import org.apache.drill.common.exceptions.DrillRuntimeException; import org.apache.drill.common.exceptions.UserException; import org.apache.drill.common.exceptions.UserRemoteException; @@ -52,13 +48,14 @@ import org.apache.drill.exec.store.sys.PersistentStore; import org.apache.drill.exec.store.sys.PersistentStoreProvider; import org.apache.drill.exec.work.EndpointListener; - -import com.carrotsearch.hppc.IntObjectHashMap; -import com.carrotsearch.hppc.predicates.IntObjectPredicate; import org.apache.drill.shaded.guava.com.google.common.base.Preconditions; import org.apache.drill.shaded.guava.com.google.common.collect.Lists; import org.apache.drill.shaded.guava.com.google.common.collect.Maps; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; + /** * Each Foreman holds its own QueryManager. This manages the events associated with execution of a particular query across all fragments. */ @@ -564,15 +561,15 @@ public DrillbitStatusListener getDrillbitStatusListener() { private final DrillbitStatusListener drillbitStatusListener = new DrillbitStatusListener() { @Override - public void drillbitRegistered(final Set registeredDrillbits) { + public void drillbitRegistered(Map registeredDrillbitsUUID) { } @Override - public void drillbitUnregistered(final Set unregisteredDrillbits) { + public void drillbitUnregistered(Map unregisteredDrillbitsUUID) { final StringBuilder failedNodeList = new StringBuilder(); boolean atLeastOneFailure = false; - for (final DrillbitEndpoint ep : unregisteredDrillbits) { + for (final DrillbitEndpoint ep : unregisteredDrillbitsUUID.keySet()) { final NodeTracker tracker = nodeMap.get(ep); if (tracker == null) { continue; // fragments were not assigned to this Drillbit diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/DefaultResourceManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/DefaultResourceManager.java index 892fa7d8b27..07d999c25c0 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/DefaultResourceManager.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/DefaultResourceManager.java @@ -19,9 +19,8 @@ import org.apache.drill.common.config.DrillConfig; import org.apache.drill.exec.ops.QueryContext; -import org.apache.drill.exec.planner.fragment.QueryParallelizer; import org.apache.drill.exec.planner.fragment.DefaultParallelizer; -import org.apache.drill.exec.proto.UserBitShared; +import org.apache.drill.exec.planner.fragment.QueryParallelizer; import org.apache.drill.exec.resourcemgr.NodeResources; import org.apache.drill.exec.resourcemgr.config.QueryQueueConfig; import org.apache.drill.exec.resourcemgr.config.exception.QueueSelectionException; @@ -38,6 +37,38 @@ public class DefaultResourceManager implements ResourceManager { + public final long memoryPerNode; + + public final int cpusPerNode; + + public DefaultResourceManager() { + memoryPerNode = DrillConfig.getMaxDirectMemory(); + + // Note: CPUs are not yet used, they will be used in a future + // enhancement. + + cpusPerNode = Runtime.getRuntime().availableProcessors(); + } + + @Override + public long memoryPerNode() { return memoryPerNode; } + + @Override + public int cpusPerNode() { return cpusPerNode; } + + @Override + public QueryResourceManager newQueryRM(final Foreman foreman) { + return new DefaultQueryResourceManager(this, foreman); + } + + public void addToWaitingQueue(final QueryResourceManager queryRM) { + throw new UnsupportedOperationException("For Default ResourceManager there shouldn't be any query in waiting " + + "queue"); + } + + @Override + public void close() { } + public static class DefaultQueryResourceManager implements QueryResourceManager { private final DefaultResourceManager rm; private final QueryContext queryContext; @@ -49,12 +80,12 @@ public DefaultQueryResourceManager(final DefaultResourceManager rm, final Forema @Override public void setCost(double cost) { - // Nothing to do by default. + // no-op } @Override public void setCost(Map costOnAssignedEndpoints) { - // Nothing to do by default + throw new UnsupportedOperationException("DefaultResourceManager doesn't support setting up cost"); } @Override @@ -67,20 +98,22 @@ public QueryAdmitResponse admit() { return QueryAdmitResponse.ADMITTED; } - public boolean reserveResources(QueryQueueConfig selectedQueue, UserBitShared.QueryId queryId) throws Exception { + public boolean reserveResources() throws Exception { + // Resource reservation is not done in this case only estimation is assigned to operator during planning time return true; } @Override public QueryQueueConfig selectQueue(NodeResources maxNodeResource) throws QueueSelectionException { - throw new UnsupportedOperationException("Queue is not supported in default resource manager"); + throw new UnsupportedOperationException("DefaultResourceManager doesn't support any queues"); } @Override public String getLeaderId() { - throw new UnsupportedOperationException("Leader is not supported in the DefaultResourceManager"); + throw new UnsupportedOperationException("DefaultResourceManager doesn't support leaders"); } + @Override public void updateState(QueryRMState newState) { // no op since Default QueryRM doesn't have any state machine } @@ -106,35 +139,4 @@ public long minimumOperatorMemory() { return 0; } } - - public final long memoryPerNode; - public final int cpusPerNode; - - public DefaultResourceManager() { - memoryPerNode = DrillConfig.getMaxDirectMemory(); - - // Note: CPUs are not yet used, they will be used in a future - // enhancement. - - cpusPerNode = Runtime.getRuntime().availableProcessors(); - } - - @Override - public long memoryPerNode() { return memoryPerNode; } - - @Override - public int cpusPerNode() { return cpusPerNode; } - - @Override - public QueryResourceManager newQueryRM(final Foreman foreman) { - return new DefaultQueryResourceManager(this, foreman); - } - - public void addToWaitingQueue(final QueryResourceManager queryRM) { - throw new UnsupportedOperationException("For Default ResourceManager there shouldn't be any query in waiting " + - "queue"); - } - - @Override - public void close() { } } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/DistributedResourceManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/DistributedResourceManager.java index 5b4c4f9a4d3..c0cdefe21aa 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/DistributedResourceManager.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/DistributedResourceManager.java @@ -17,14 +17,19 @@ */ package org.apache.drill.exec.work.foreman.rm; +import avro.shaded.com.google.common.annotations.VisibleForTesting; +import org.apache.drill.common.DrillNode; import org.apache.drill.common.config.DrillConfig; import org.apache.drill.common.exceptions.DrillRuntimeException; import org.apache.drill.exec.ExecConstants; +import org.apache.drill.exec.coord.zk.ZKClusterCoordinator; import org.apache.drill.exec.exception.StoreException; import org.apache.drill.exec.ops.QueryContext; import org.apache.drill.exec.planner.fragment.DistributedQueueParallelizer; import org.apache.drill.exec.planner.fragment.QueryParallelizer; -import org.apache.drill.exec.proto.UserBitShared; +import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint; +import org.apache.drill.exec.proto.UserBitShared.QueryResult.QueryState; +import org.apache.drill.exec.proto.helper.QueryIdHelper; import org.apache.drill.exec.resourcemgr.NodeResources; import org.apache.drill.exec.resourcemgr.config.QueryQueueConfig; import org.apache.drill.exec.resourcemgr.config.RMCommonDefaults; @@ -37,19 +42,25 @@ import org.apache.drill.exec.resourcemgr.rmblobmgr.RMConsistentBlobStoreManager; import org.apache.drill.exec.resourcemgr.rmblobmgr.exception.ResourceUnavailableException; import org.apache.drill.exec.server.DrillbitContext; +import org.apache.drill.exec.work.foreman.DrillbitStatusListener; import org.apache.drill.exec.work.foreman.Foreman; import org.apache.drill.exec.work.foreman.rm.QueryQueue.QueryQueueException; import org.apache.drill.exec.work.foreman.rm.QueryQueue.QueueTimeoutException; import org.apache.drill.shaded.guava.com.google.common.base.Preconditions; import org.apache.drill.shaded.guava.com.google.common.base.Stopwatch; +import java.lang.reflect.Constructor; import java.util.Collection; import java.util.Comparator; -import java.util.HashMap; import java.util.Map; import java.util.PriorityQueue; +import java.util.Queue; import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; import static org.apache.drill.exec.ExecConstants.RM_WAIT_THREAD_INTERVAL; @@ -61,31 +72,35 @@ public class DistributedResourceManager implements ResourceManager { private final DrillbitContext context; - private final DrillConfig rmConfig; - public final long memoryPerNode; - public final int cpusPerNode; + private final int cpusPerNode; + + private final Thread waitQueueThread; - private final WaitQueueThread waitQueueThread; + private volatile AtomicBoolean exitDaemonThreads = new AtomicBoolean(false); private final RMBlobStoreManager rmBlobStoreManager; // Wait queues for each queue which holds queries that are admitted by leader but not yet executed because resource // is unavailable - private final Map> waitingQueuesForAdmittedQuery = new HashMap<>(); + private final Map> waitingQueuesForAdmittedQuery = new ConcurrentHashMap<>(); // Comparator used in priority max-wait queues for each queue such that query which came in first is at the top of // the queue. Query which came first will have highest elapsed time private static final Comparator waitTimeComparator = (DistributedQueryRM d1, DistributedQueryRM d2) -> Long.compare(d2.elapsedWaitTime(), d1.elapsedWaitTime()); + private final Queue queryRMCleanupQueue = new ConcurrentLinkedQueue<>(); + + private final Thread queryRMCleanupThread; + public DistributedResourceManager(DrillbitContext context) throws DrillRuntimeException { - memoryPerNode = DrillConfig.getMaxDirectMemory(); - cpusPerNode = Runtime.getRuntime().availableProcessors(); try { + memoryPerNode = DrillConfig.getMaxDirectMemory(); + cpusPerNode = Runtime.getRuntime().availableProcessors(); this.context = context; - this.rmConfig = DrillConfig.createForRM(); + final DrillConfig rmConfig = DrillConfig.createForRM(); rmPoolTree = new ResourcePoolTreeImpl(rmConfig, DrillConfig.getMaxDirectMemory(), Runtime.getRuntime().availableProcessors(), 1); logger.debug("Successfully parsed RM config \n{}", rmConfig.getConfig(ResourcePoolTreeImpl.ROOT_POOL_CONFIG_KEY)); @@ -93,13 +108,19 @@ public DistributedResourceManager(DrillbitContext context) throws DrillRuntimeEx for (String leafQueue : leafQueues) { waitingQueuesForAdmittedQuery.put(leafQueue, new PriorityQueue<>(waitTimeComparator)); } - this.rmBlobStoreManager = new RMConsistentBlobStoreManager(context, rmPoolTree.getAllLeafQueues().values()); + this.rmBlobStoreManager = new RMConsistentBlobStoreManager(context, rmPoolTree); - // start the wait thread + // Register the DrillbitStatusListener which registers the localBitResourceShare + context.getClusterCoordinator().addDrillbitStatusListener( + new RegisterLocalBitResources(context, rmPoolTree, rmBlobStoreManager)); + + // calculate wait interval final int waitThreadInterval = calculateWaitInterval(rmConfig, rmPoolTree.getAllLeafQueues().values()); logger.debug("Wait thread refresh interval is set as {}", waitThreadInterval); - this.waitQueueThread = new WaitQueueThread(waitThreadInterval); - this.waitQueueThread.setDaemon(true); + // start the wait thread + this.waitQueueThread = startDaemonThreads(WaitQueueThread.class, waitThreadInterval); + // start the cleanup thread + queryRMCleanupThread = startDaemonThreads(CleanupThread.class, waitThreadInterval); } catch (RMConfigException ex) { throw new DrillRuntimeException(String.format("Failed while parsing Drill RM Configs. Drillbit won't be started" + " unless config is fixed or RM is disabled by setting %s to false", ExecConstants.RM_ENABLED), ex); @@ -108,22 +129,6 @@ public DistributedResourceManager(DrillbitContext context) throws DrillRuntimeEx } } - private int calculateWaitInterval(DrillConfig rmConfig, Collection leafQueues) { - if (rmConfig.hasPath(RM_WAIT_THREAD_INTERVAL)) { - return rmConfig.getInt(RM_WAIT_THREAD_INTERVAL); - } - - // Otherwise out of all the configured queues use half of the minimum positive wait time as the interval - int minWaitInterval = RMCommonDefaults.MAX_WAIT_TIMEOUT_IN_MS; - for (QueryQueueConfig leafQueue : leafQueues) { - int queueWaitTime = leafQueue.getWaitTimeoutInMs(); - if (queueWaitTime > 0) { - minWaitInterval = Math.min(minWaitInterval, queueWaitTime); - } - } - return minWaitInterval; - } - @Override public long memoryPerNode() { return memoryPerNode; @@ -139,52 +144,126 @@ public QueryResourceManager newQueryRM(Foreman foreman) { return new DistributedQueryRM(this, foreman); } + @VisibleForTesting + public ResourcePoolTree getRmPoolTree() { + return rmPoolTree; + } + @Override - public void addToWaitingQueue(final QueryResourceManager queryRM) { - final DistributedQueryRM distributedQueryRM = (DistributedQueryRM)queryRM; - final String queueName = distributedQueryRM.queueName(); - final PriorityQueue waitingQueue = waitingQueuesForAdmittedQuery.get(queueName); - waitingQueue.add(distributedQueryRM); + public void close() { + // interrupt the wait thread + exitDaemonThreads.set(true); + waitQueueThread.interrupt(); + queryRMCleanupThread.interrupt(); + + // Clear off the QueryRM for admitted queries which are in waiting state. This should be fine even in case of + // graceful shutdown since other bits will get notification as bit going down and will update the cluster state + // accordingly + // TODO: Fix race condition between wait thread completing to process waitQueryRM and again putting back the + // object in the queue. In parallel close thread clearing off the queue + waitingQueuesForAdmittedQuery.clear(); } - private void reserveResources(Map queryResourceAssignment, - QueryQueueConfig selectedQueue, String leaderId, String queryId, - String foremanNode) throws Exception { - //rmBlobStoreManager.reserveResources(); + /** + * Calculates the refresh interval for the wait thread which process all the admitted queries by leader but are + * waiting on Foreman node for resource availability. If all the queues wait timeout is set to 0 then there won't + * be any queries in the wait queue and refresh interval will be half of MAX_WAIT_TIMEOUT. Otherwise it will be + * half of minimum of waiting time across all queues. + * @param rmConfig rm configurations + * @param leafQueues configured collection of leaf pools or queues + * @return refresh interval for wait thread + */ + private int calculateWaitInterval(DrillConfig rmConfig, Collection leafQueues) { + if (rmConfig.hasPath(RM_WAIT_THREAD_INTERVAL)) { + return rmConfig.getInt(RM_WAIT_THREAD_INTERVAL); + } + + // Otherwise out of all the configured queues use half of the minimum positive wait time as the interval + int minWaitInterval = RMCommonDefaults.MAX_WAIT_TIMEOUT_IN_MS; + for (QueryQueueConfig leafQueue : leafQueues) { + int queueWaitTime = leafQueue.getWaitTimeoutInMs(); + if (queueWaitTime > 0) { + minWaitInterval = Math.min(minWaitInterval, queueWaitTime); + } + } + final int halfMinWaitInterval = minWaitInterval / 2; + return (halfMinWaitInterval == 0) ? minWaitInterval : halfMinWaitInterval; } - private void freeResources(Map queryResourceAssignment, QueryQueueConfig selectedQueue, - String leaderId, String queryId, String foremanNode) throws Exception { + private Thread startDaemonThreads(Class threadClass, Integer interval) { + try { + final Constructor threadConstructor = threadClass.getConstructor(DistributedResourceManager.class, Integer.class); + final Thread threadToCreate = (Thread) threadConstructor.newInstance(this, interval); + threadToCreate.setDaemon(true); + threadToCreate.start(); + return threadToCreate; + } catch (Exception ex) { + throw new DrillRuntimeException(String.format("Failed to create %s daemon thread for Distributed RM", + threadClass.getName()), ex); + } + } + private void addToWaitingQueue(final QueryResourceManager queryRM) { + final DistributedQueryRM distributedQueryRM = (DistributedQueryRM)queryRM; + final String queueName = distributedQueryRM.queueName(); + synchronized (waitingQueuesForAdmittedQuery) { + final PriorityQueue waitingQueue = waitingQueuesForAdmittedQuery.get(queueName); + waitingQueue.add(distributedQueryRM); + logger.info("Count of times queryRM for the query {} is added in the wait queue is {}", + ((DistributedQueryRM) queryRM).queryIdString, distributedQueryRM.incrementAndGetWaitRetryCount()); + } } - public ResourcePoolTree getRmPoolTree() { - return rmPoolTree; + private void reserveResources(Map queryResourceAssignment, + QueryQueueConfig selectedQueue, String leaderId, String queryId, + String foremanUUID) throws Exception { + logger.info("Reserving resources for query {}. [Details: ResourceMap: {}]", queryId, + queryResourceAssignment.toString()); + rmBlobStoreManager.reserveResources(queryResourceAssignment, selectedQueue, leaderId, queryId, foremanUUID); } - @Override - public void close() { + private String freeResources(Map queryResourceAssignment, QueryQueueConfig selectedQueue, + String leaderId, String queryId, String foremanUUID) throws Exception { + logger.info("Free resources for query {}. [Details: ResourceMap: {}]", queryId, queryResourceAssignment.toString()); + return rmBlobStoreManager.freeResources(queryResourceAssignment, selectedQueue, leaderId, queryId, foremanUUID); } - public class DistributedQueryRM implements QueryResourceManager { + public static class DistributedQueryRM implements QueryResourceManager { private final DistributedResourceManager drillRM; - private final QueryContext context; + private final QueryContext queryContext; private final Foreman foreman; + private final String foremanUUID; + + private final String queryIdString; + private QueryRMState currentState; private Stopwatch waitStartTime; private Map assignedEndpointsCost; + private QueryQueueConfig selectedQueue; + + private String admittedLeaderUUID; + + private String currentQueueLeader; + + private int cleanupTryCount; + + private int retryCountAfterWaitQueue; + DistributedQueryRM(ResourceManager resourceManager, Foreman queryForeman) { + Preconditions.checkArgument(resourceManager instanceof DistributedResourceManager); this.drillRM = (DistributedResourceManager) resourceManager; - this.context = queryForeman.getQueryContext(); + this.queryContext = queryForeman.getQueryContext(); this.foreman = queryForeman; + this.queryIdString = QueryIdHelper.getQueryId(queryContext.getQueryId()); currentState = QueryRMState.STARTED; + foremanUUID = queryContext.getOnlineEndpointNodeUUIDs().get(queryContext.getCurrentEndpointNode()); } @Override @@ -205,57 +284,69 @@ public void setCost(Map costOnAssignedEndpoints) { } public long queryMemoryPerNode() { - return 0; + Preconditions.checkState(selectedQueue != null, "Queue is not yet selected for this query"); + return selectedQueue.getMaxQueryMemoryInMBPerNode(); } @Override public long minimumOperatorMemory() { - return 0; + return queryContext.getOption(ExecConstants.MIN_MEMORY_PER_BUFFERED_OP_KEY).num_val; } @Override public QueryParallelizer getParallelizer(boolean planHasMemory) { // currently memory planning is disabled. Enable it once the RM functionality is fully implemented. - return new DistributedQueueParallelizer(true || planHasMemory, this.context, this); + return new DistributedQueueParallelizer(planHasMemory, this.queryContext, this); } @Override public QueryAdmitResponse admit() throws QueueTimeoutException, QueryQueueException { - // TODO: for now it will just return since leader election is not available + // TODO: for now it will just return ADMITTED since leader election is not available // Once leader election support is there we will throw exception in case of error // otherwise just return + Preconditions.checkState(selectedQueue != null, "Query is being admitted before selecting " + + "a queue for it"); updateState(QueryRMState.ENQUEUED); return QueryAdmitResponse.ADMITTED; } @Override public String queueName() { - return ""; + Preconditions.checkState(selectedQueue != null, "Queue is not selected yet"); + return selectedQueue.getQueueName(); } @Override public QueryQueueConfig selectQueue(NodeResources maxNodeResource) throws QueueSelectionException { - return drillRM.rmPoolTree.selectOneQueue(context, maxNodeResource); - //TODO: based on selected queue store the leader UUID as well + if (selectedQueue != null) { + return selectedQueue; + } + + selectedQueue = drillRM.rmPoolTree.selectOneQueue(queryContext, maxNodeResource); + // TODO: Set the LeaderUUID based on the selected queue + admittedLeaderUUID = foremanUUID; + currentQueueLeader = admittedLeaderUUID; + logger.info("Selected queue {} for query {} with leader {}", selectedQueue.getQueueName(), queryIdString, + admittedLeaderUUID); + return selectedQueue; } @Override public String getLeaderId() { - // TODO: Return emoty string for now - return ""; + return admittedLeaderUUID; } - public boolean reserveResources(QueryQueueConfig selectedQueue, UserBitShared.QueryId queryId) throws Exception { + public boolean reserveResources() throws Exception { try { + Preconditions.checkState(selectedQueue != null, "A queue is not selected for the query " + + "before trying to reserve resources for this query"); Preconditions.checkState(assignedEndpointsCost != null, "Cost of the query is not set before calling reserve resources"); - // TODO: pass the correct parameter values to function below - drillRM.reserveResources(null, null, null, null, null); + drillRM.reserveResources(assignedEndpointsCost, selectedQueue, admittedLeaderUUID, queryIdString, foremanUUID); updateState(QueryRMState.RESERVED_RESOURCES); return true; } catch (ResourceUnavailableException ex) { - // add the query to the waiting queue for retry - // set the wait time if not already done + // add the query to the waiting queue for retry and set the wait time if not already done if (waitStartTime == null) { waitStartTime = Stopwatch.createStarted(); } @@ -265,7 +356,7 @@ public boolean reserveResources(QueryQueueConfig selectedQueue, UserBitShared.Qu // timeout has expired so don't put in waiting queue throw new QueueWaitTimeoutExpired(String.format("Failed to reserve resources for the query and the wait " + "timeout is also expired. [Details: QueryId: %s, Queue: %s, ElapsedTime: %d", - queryId, selectedQueue.getQueueName(), timeElapsedWaiting), ex); + queryIdString, selectedQueue.getQueueName(), timeElapsedWaiting), ex); } drillRM.addToWaitingQueue(this); return false; @@ -275,17 +366,53 @@ public boolean reserveResources(QueryQueueConfig selectedQueue, UserBitShared.Qu } } - private long elapsedWaitTime() { - return waitStartTime.elapsed(TimeUnit.MILLISECONDS); + @Override + public void updateState(QueryRMState newState) { + boolean isSuccessful = false; + switch (currentState) { + case STARTED: + isSuccessful = (newState == QueryRMState.ENQUEUED || newState == QueryRMState.FAILED); + break; + case ENQUEUED: + isSuccessful = (newState == QueryRMState.ADMITTED || newState == QueryRMState.FAILED); + break; + case ADMITTED: + isSuccessful = (newState == QueryRMState.RESERVED_RESOURCES || newState == QueryRMState.DEQUEUED); + break; + case RESERVED_RESOURCES: + isSuccessful = (newState == QueryRMState.RELEASED_RESOURCES); + break; + case RELEASED_RESOURCES: + isSuccessful = (newState == QueryRMState.DEQUEUED); + break; + case DEQUEUED: + isSuccessful = (newState == QueryRMState.COMPLETED); + break; + } + + final String logString = String.format("QueryRM state transition from %s --> %s is %s", + currentState.toString(), newState.toString(), isSuccessful ? "successful" : "failed"); + if (isSuccessful) { + this.currentState = newState; + logger.info(logString); + return; + } + + throw new IllegalStateException(logString); } - public void updateState(QueryRMState newState) { - // no op since Default QueryRM doesn't have any state machine - // for now we are just overwriting the currentState. May be we can add logic for handling incorrect - // state transitions and allowed state transitions - this.currentState = newState; + @VisibleForTesting + public QueryRMState getCurrentState() { + return currentState; } + /** + * Exit on queryRM will only be called from ForemanResult::close() in case when query either fails or completes + * or is cancelled. + * When query fails/completes/cancel then it will never be in the wait queue. Hence exit should not worry about + * removing the queryRM object from wait queue. + * TODO: Incomplete because protocol to send message to leader is unavailable + */ @Override public void exit() { // 1. if queryRM is in admitted state: That means exit is called either when query is failed. When query is @@ -305,28 +432,193 @@ public void exit() { // // 2. if query is in reserved resources state then update zookeeper to release resources and send message back to // current leader to release the slot. + + switch (currentState) { + case ADMITTED: + // send message to admittedQueueLeader about completion of this query so that it can release it's local queue + // slot. This send should be a sync call. If send of message fails then add this query back to + // queryRMCleanupQueue. If send failure happens because of leader change then ignore the failure + updateState(QueryRMState.DEQUEUED); + break; + case RESERVED_RESOURCES: + // try to release the resources and update state on Zookeeper + try { + currentQueueLeader = drillRM.freeResources(assignedEndpointsCost, selectedQueue, admittedLeaderUUID, + queryIdString, foremanUUID); + // successfully released resources so update the state + updateState(QueryRMState.RELEASED_RESOURCES); + } catch (Exception ex) { + logger.info("Failed while freeing resources for this query {} in queryRM exit for {} time", queryIdString, + incrementAndGetCleanupCount()); + drillRM.queryRMCleanupQueue.add(this); + return; + } + case RELEASED_RESOURCES: + // send message to currentQueueLeader about completion of this query so that it can release it's local queue + // slot. This send should be a sync call. If send of message fails then add this query back to + // queryRMCleanupQueue. If send failure happens because of leader change then ignore the failure + updateState(QueryRMState.DEQUEUED); + break; + case STARTED: + case ENQUEUED: + Preconditions.checkState(foreman.getState() == QueryState.FAILED, "QueryRM exit is " + + "called in an unexpected query state. [Details: QueryRM state: %s, Query State: %s]", + currentState, foreman.getState()); + updateState(QueryRMState.FAILED); + return; + default: + throw new IllegalStateException("QueryRM exit is called in unexpected state. Looks like something is wrong " + + "with internal state!!"); + } + updateState(QueryRMState.COMPLETED); + } + + private long elapsedWaitTime() { + return waitStartTime.elapsed(TimeUnit.MILLISECONDS); + } + + private int incrementAndGetCleanupCount() { + ++cleanupTryCount; + return cleanupTryCount; + } + + private int incrementAndGetWaitRetryCount() { + ++retryCountAfterWaitQueue; + return retryCountAfterWaitQueue; } } - public static class WaitQueueThread extends Thread { + /** + * All queries which are in admitted state but are not able to reserve resources will be in this queue and process + * by the wait thread. When query is in wait thread in can never fail since it's not running and cancellation will + * wait for it to go in running state. + */ + private class WaitQueueThread extends Thread { private final int refreshInterval; - public WaitQueueThread(int waitInterval) { - setName("DistributedResourceManager.WaitThread"); + public WaitQueueThread(Integer waitInterval) { refreshInterval = waitInterval; + setName("DistributedResourceManager.WaitThread"); } + // TODO: Incomplete @Override public void run() { - while (true) { + while (!exitDaemonThreads.get()) { try { + synchronized (waitingQueuesForAdmittedQuery) { + for (PriorityQueue queue : waitingQueuesForAdmittedQuery.values()) { + // get the initial queue count such that we only try to dequeue that many query only since newly dequeued + // query can also meanwhile come back to this queue. + final int queueSize = queue.size(); + while(queueSize > 0) { + final DistributedQueryRM queryRM = queue.poll(); + context.getExecutor().submit(queryRM.foreman); + } + } + } Thread.sleep(refreshInterval); } catch (InterruptedException ex) { logger.error("Thread {} is interrupted", getName()); - Thread.currentThread().interrupt(); - break; } } } } + + /** + * All the completed queries whose result is sent back to client but during cleanup encountered some issues will be + * present in the queryRMCleanupQueue for the lifetime of this Drillbit. These queryRM object will be tried for + * cleanup since that affect the state of the cluster + */ + private class CleanupThread extends Thread { + private final int refreshTime; + + public CleanupThread(Integer refreshInterval) { + this.refreshTime = refreshInterval; + setName("DistributedResourceManager.CleanupThread"); + } + + @Override + public void run() { + while(!exitDaemonThreads.get()) { + try { + int queryRMCount = queryRMCleanupQueue.size(); + + while (queryRMCount > 0) { + --queryRMCount; + final DistributedQueryRM queryRM = queryRMCleanupQueue.poll(); + queryRM.exit(); + } + + // wait here for some time + Thread.sleep(refreshTime); + } catch (InterruptedException ex) { + logger.error("Thread {} is interrupted", getName()); + } + } + } + } + + public static class RegisterLocalBitResources implements DrillbitStatusListener { + + private final DrillNode localEndpointNode; + + private final RMBlobStoreManager storeManager; + + private final NodeResources localBitResourceShare; + + private Set leafQueues; + + private final ZKClusterCoordinator coord; + + private final DrillbitContext context; + + public RegisterLocalBitResources(DrillbitContext context, ResourcePoolTree rmPoolTree, + RMBlobStoreManager storeManager) { + this.localEndpointNode = DrillNode.create(context.getEndpoint()); + this.localBitResourceShare = rmPoolTree.getRootPoolResources(); + this.storeManager = storeManager; + this.coord = (ZKClusterCoordinator) context.getClusterCoordinator(); + this.context = context; + this.leafQueues = rmPoolTree.getAllLeafQueues().keySet(); + } + + @Override + public void drillbitUnregistered(Map unregisteredDrillbitsUUID) { + // no-op for now. May be we can use this to handler failure scenarios of bit going down + } + + @Override + public void drillbitRegistered(Map registeredDrillbitsUUID) { + // Check if in registeredDrillbits local drillbit is present and with state as ONLINE since this listener + // will be invoked for every state change as well + // TODO: This can be improved once DrillNode is used everywhere instead of DrillbitEndpoint + final Map registeredNodeUUID = registeredDrillbitsUUID.entrySet().stream() + .collect(Collectors.toMap(x -> DrillNode.create(x.getKey()), Map.Entry::getValue)); + final Map uuidToEndpoint = registeredDrillbitsUUID.entrySet().stream() + .collect(Collectors.toMap(Map.Entry::getValue, Map.Entry::getKey)); + + try { + if (registeredNodeUUID.containsKey(localEndpointNode)) { + final String localBitUUID = registeredNodeUUID.get(localEndpointNode); + final DrillbitEndpoint localEndpoint = uuidToEndpoint.get(localBitUUID); + + if (localEndpoint.getState() == DrillbitEndpoint.State.ONLINE) { + storeManager.registerResource(localBitUUID, localBitResourceShare); + logger.info("Registering local bit resource share"); + + // TODO: Temp update queue leaders as self + for (String queueName : leafQueues) { + storeManager.updateLeadershipInformation(queueName, localBitUUID); + } + } + } + } catch (Exception ex) { + // fails to register local bit resources to zookeeper + logger.error("Fails to register local bit resource share so unregister local Drillbit"); + // below getRegistrationHandle blocks until registration handle is set, if already set then return immediately + coord.unregister(context.getRegistrationHandle()); + } + } + } } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/DynamicResourceManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/DynamicResourceManager.java index 872ef50a244..391afa8c5cc 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/DynamicResourceManager.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/DynamicResourceManager.java @@ -66,11 +66,6 @@ public synchronized QueryResourceManager newQueryRM(Foreman foreman) { return activeRm.newQueryRM(foreman); } - @Override - public void addToWaitingQueue(QueryResourceManager queryRM) { - throw new UnsupportedOperationException("Dynamic Resource Manager needs to be retired"); - } - private void refreshRM() { long now = System.currentTimeMillis(); if (now < nextUpdateTime) { diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/QueryResourceManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/QueryResourceManager.java index ef61bf39842..b82fa9bb6e4 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/QueryResourceManager.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/QueryResourceManager.java @@ -18,7 +18,6 @@ package org.apache.drill.exec.work.foreman.rm; import org.apache.drill.exec.planner.fragment.QueryParallelizer; -import org.apache.drill.exec.proto.UserBitShared; import org.apache.drill.exec.resourcemgr.NodeResources; import org.apache.drill.exec.resourcemgr.config.QueryQueueConfig; import org.apache.drill.exec.resourcemgr.config.exception.QueueSelectionException; @@ -27,12 +26,11 @@ import java.util.Map; -/** - * Extends a {@link QueryResourceAllocator} to provide queueing support. - */ - public interface QueryResourceManager { + /** + * Responses which {@link QueryResourceManager#admit()} method can return to the caller + */ enum QueryAdmitResponse { UNKNOWN, ADMITTED, @@ -44,6 +42,9 @@ public String toString() { } } + /** + * State machine of the QueryResourceManager + */ enum QueryRMState { STARTED, ENQUEUED, @@ -51,6 +52,7 @@ enum QueryRMState { RESERVED_RESOURCES, RELEASED_RESOURCES, DEQUEUED, + FAILED, COMPLETED; @Override @@ -64,7 +66,6 @@ public String toString() { * to short-circuit expensive logic if no queuing will actually * be done. This is a static attribute per Drillbit run. */ - boolean hasQueue(); /** @@ -73,42 +74,50 @@ public String toString() { * to place the job into the correct queue. * @param cost */ - void setCost(double cost); + /** + * Cost of query in terms of DrillbitEndpoint UUID to resources required by all minor fragments of this query + * which will run on that DrillbitEndpoint. {@link QueryParallelizer} calculates this costs based on it's own + * heuristics for each query and sets it for the ResourceManager. + * @param costOnAssignedEndpoints map of DrillbitEndpointUUID to resources required by this query on that node + */ void setCost(Map costOnAssignedEndpoints); + /** * Create a parallelizer to parallelize each major fragment of the query into * many minor fragments. The parallelizer encapsulates the logic of how much * memory and parallelism is required for the query. * @param memoryPlanning memory planning needs to be done during parallelization - * @return + * @return {@link QueryParallelizer} to use */ QueryParallelizer getParallelizer(boolean memoryPlanning); /** - * Admit the query into the cluster. Blocks until the query - * can run. (Later revisions may use a more thread-friendly - * approach.) - * @throws QueryQueueException if something goes wrong with the - * queue mechanism + * Admit the query into the cluster. Can be sync or async call which depends upon the implementation. Caller should + * use returned response to take necessary action + * @return {@link QueryAdmitResponse} response for the admit call + * @throws QueryQueueException if something goes wrong with the queue mechanism + * @throws QueueTimeoutException if admit requests times out */ - QueryAdmitResponse admit() throws QueueTimeoutException, QueryQueueException; - /** - * Returns the name of the queue (if any) on which the query was - * placed. Valid only after the query is admitted. - * - * @return queue name, or null if queuing is not enabled. + * Returns the name of the queue (if any) on which the query was placed. + * @return queue name, or null if queue is not supported */ - String queueName(); - + /** + * @return max memory a query can use on a node + */ long queryMemoryPerNode(); + /** + * TODO: Try to deprecate this api since it's only used by ThrottledResourceManager. It can be replaced by per + * operator minimum memory which will be added with DistributedResourceManager support. + * @return minimum memory required by buffered operator + */ long minimumOperatorMemory(); /** @@ -118,12 +127,26 @@ public String toString() { void updateState(QueryRMState newState); /** - * Called to reserve resources required by query. Updates the queryRM state to RESERVED_RESOURCES if successful + * Called to reserve resources required by query in a state store. This will help to make decisions for future queries + * based on the information in state store about the available resources in the cluster. + * @return true successfully reserved resources, false failure while reserving resources + * @throws Exception in case of non transient failure */ - boolean reserveResources(QueryQueueConfig selectedQueue, UserBitShared.QueryId queryId) throws Exception; + boolean reserveResources() throws Exception; + /** + * Select a queue out of all configured ones for this query. The selected queue config will be later used to make + * decisions about resource assignment to this query. + * @param maxNodeResource maximum resources which this query needs across all assigned endpoints + * @return configuration of selected queue for this query + * @throws QueueSelectionException + */ QueryQueueConfig selectQueue(NodeResources maxNodeResource) throws QueueSelectionException; + /** + * TODO: Check if this api is needed ? + * @return leader of selected queue to which admit request will be sent + */ String getLeaderId(); /** * Mark the query as completing, giving up its slot in the diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/ResourceManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/ResourceManager.java index ac9554971dc..821bbf2f2fb 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/ResourceManager.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/ResourceManager.java @@ -51,7 +51,5 @@ public interface ResourceManager { QueryResourceManager newQueryRM(final Foreman foreman); - void addToWaitingQueue(final QueryResourceManager queryRM); - void close(); } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/ResourceManagerBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/ResourceManagerBuilder.java index ec1f2c198d1..435516659f9 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/ResourceManagerBuilder.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/ResourceManagerBuilder.java @@ -18,9 +18,11 @@ package org.apache.drill.exec.work.foreman.rm; import org.apache.drill.common.config.DrillConfig; +import org.apache.drill.exec.ExecConstants; import org.apache.drill.exec.coord.ClusterCoordinator; import org.apache.drill.exec.coord.local.LocalClusterCoordinator; import org.apache.drill.exec.server.DrillbitContext; +import org.apache.drill.exec.server.options.SystemOptionManager; /** * Builds the proper resource manager and queue implementation for the configured @@ -60,17 +62,23 @@ public ResourceManagerBuilder(final DrillbitContext context) { } public ResourceManager build() { - ClusterCoordinator coord = context.getClusterCoordinator(); - DrillConfig config = context.getConfig(); + final ClusterCoordinator coord = context.getClusterCoordinator(); + final DrillConfig config = context.getConfig(); + final SystemOptionManager systemOptions = context.getOptionManager(); if (coord instanceof LocalClusterCoordinator) { if (config.getBoolean(EmbeddedQueryQueue.ENABLED)) { - logger.debug("Enabling embedded, local query queue."); + logger.info("Enabling embedded, local query queue"); return new ThrottledResourceManager(context, new EmbeddedQueryQueue(context)); } else { - logger.debug("No query queueing enabled."); + logger.info("Zookeeper is not configured as ClusterCoordinator hence using Default Manager. [Details: " + + "isRMEnabled: {}]", config.getBoolean(ExecConstants.RM_ENABLED)); return new DefaultResourceManager(); } + } else if (config.getBoolean(ExecConstants.RM_ENABLED) && !systemOptions.getOption(ExecConstants.ENABLE_QUEUE)){ + logger.info("RM is enabled and queues are disabled so using Distributed Resource Manager"); + return new DistributedResourceManager(context); } else { + logger.info("Using Dynamic Resource Manager to either enable Default of Throttled Resource Manager"); return new DynamicResourceManager(context); } } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/ThrottledResourceManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/ThrottledResourceManager.java index 9f7ff6fb77f..05ab953e47f 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/ThrottledResourceManager.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/ThrottledResourceManager.java @@ -20,7 +20,6 @@ import org.apache.drill.exec.ops.QueryContext; import org.apache.drill.exec.planner.fragment.QueryParallelizer; import org.apache.drill.exec.planner.fragment.ZKQueueParallelizer; -import org.apache.drill.exec.proto.UserBitShared; import org.apache.drill.exec.resourcemgr.NodeResources; import org.apache.drill.exec.resourcemgr.config.QueryQueueConfig; import org.apache.drill.exec.resourcemgr.config.exception.QueueSelectionException; @@ -53,6 +52,35 @@ public class ThrottledResourceManager extends AbstractResourceManager { private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ThrottledResourceManager.class); + private final QueryQueue queue; + + public ThrottledResourceManager(final DrillbitContext drillbitContext, + final QueryQueue queue) { + super(drillbitContext); + this.queue = queue; + queue.setMemoryPerNode(memoryPerNode()); + } + + public long minimumOperatorMemory() { + return queue.minimumOperatorMemory(); + } + + public long defaultQueryMemoryPerNode(double cost) { + return queue.defaultQueryMemoryPerNode(cost); + } + + public QueryQueue queue() { return queue; } + + @Override + public QueryResourceManager newQueryRM(Foreman foreman) { + return new QueuedQueryResourceManager(this, foreman); + } + + @Override + public void close() { + queue.close(); + } + /** * Per-query resource manager. Handles resources and optional queue lease for * a single query. As such, this is a non-shared resource: it is associated @@ -115,21 +143,24 @@ public long minimumOperatorMemory() { } @Override - public boolean reserveResources(QueryQueueConfig selectedQueue, UserBitShared.QueryId queryId) throws Exception { - // no op + public boolean reserveResources() throws Exception { + // Resource reservation is not done in this case only estimation is assigned to operator during planning time return true; } @Override public QueryQueueConfig selectQueue(NodeResources maxNodeResource) throws QueueSelectionException { - throw new UnsupportedOperationException("Select queue is not supported in QueuedQueryResourceManager"); + throw new UnsupportedOperationException("QueuedQueryResourceManager supports ZKQueue not Drills distributed " + + "queue"); } @Override public String getLeaderId() { - throw new UnsupportedOperationException("Leader is not supported in QueuedQueryResourceManager"); + throw new UnsupportedOperationException("QueuedQueryResourceManager has Zookeeper as the central leader for all" + + " queues."); } + @Override public void updateState(QueryRMState state) { // no-op Doesn't support any state machine } @@ -150,39 +181,4 @@ public String queueName() { return lease == null ? null : lease.queueName(); } } - - private final QueryQueue queue; - - public ThrottledResourceManager(final DrillbitContext drillbitContext, - final QueryQueue queue) { - super(drillbitContext); - this.queue = queue; - queue.setMemoryPerNode(memoryPerNode()); - } - - public long minimumOperatorMemory() { - return queue.minimumOperatorMemory(); - } - - public long defaultQueryMemoryPerNode(double cost) { - return queue.defaultQueryMemoryPerNode(cost); - } - - public QueryQueue queue() { return queue; } - - @Override - public QueryResourceManager newQueryRM(Foreman foreman) { - return new QueuedQueryResourceManager(this, foreman); - } - - @Override - public void addToWaitingQueue(QueryResourceManager queryRM) { - // no-op - return; - } - - @Override - public void close() { - queue.close(); - } } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java index 9e9bf3ff44c..bf24bcd638d 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java @@ -17,14 +17,6 @@ */ package org.apache.drill.exec.work.fragment; -import java.io.IOException; -import java.security.PrivilegedExceptionAction; -import java.util.Queue; -import java.util.Set; -import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicReference; - import org.apache.drill.common.DeferredException; import org.apache.drill.common.EventProcessor; import org.apache.drill.common.exceptions.UserException; @@ -37,7 +29,6 @@ import org.apache.drill.exec.physical.impl.RootExec; import org.apache.drill.exec.proto.BitControl.FragmentStatus; import org.apache.drill.exec.proto.BitControl.PlanFragment; -import org.apache.drill.exec.proto.CoordinationProtos; import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint; import org.apache.drill.exec.proto.ExecProtos.FragmentHandle; import org.apache.drill.exec.proto.UserBitShared.FragmentState; @@ -49,6 +40,15 @@ import org.apache.drill.exec.work.foreman.DrillbitStatusListener; import org.apache.hadoop.security.UserGroupInformation; +import java.io.IOException; +import java.security.PrivilegedExceptionAction; +import java.util.Map; +import java.util.Queue; +import java.util.Set; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; + import static org.apache.drill.exec.server.FailureUtils.EXIT_CODE_HEAP_OOM; /** @@ -503,13 +503,14 @@ public Throwable getFailureCause(){ private class FragmentDrillbitStatusListener implements DrillbitStatusListener { @Override - public void drillbitRegistered(final Set registeredDrillbits) { + public void drillbitRegistered(Map registeredDrillbitsUUID) { } @Override - public void drillbitUnregistered(final Set unregisteredDrillbits) { + public void drillbitUnregistered(Map unregisteredDrillbitsUUID) { // if the defunct Drillbit was running our Foreman, then cancel the query final DrillbitEndpoint foremanEndpoint = FragmentExecutor.this.fragmentContext.getForemanEndpoint(); + final Set unregisteredDrillbits = unregisteredDrillbitsUUID.keySet(); if (unregisteredDrillbits.contains(foremanEndpoint)) { logger.warn("Foreman {} no longer active. Cancelling fragment {}.", foremanEndpoint.getAddress(), diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/planner/PhysicalPlanReaderTestFactory.java b/exec/java-exec/src/test/java/org/apache/drill/exec/planner/PhysicalPlanReaderTestFactory.java index 5aaeb5f1b9c..29d2199aaff 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/planner/PhysicalPlanReaderTestFactory.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/planner/PhysicalPlanReaderTestFactory.java @@ -22,7 +22,7 @@ import org.apache.drill.common.config.LogicalPlanPersistence; import org.apache.drill.common.scanner.ClassPathScanner; import org.apache.drill.common.scanner.persistence.ScanResult; -import org.apache.drill.exec.proto.CoordinationProtos; +import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint; import org.apache.drill.exec.server.DrillbitContext; import org.apache.drill.exec.store.StoragePluginRegistry; import org.junit.experimental.categories.Category; @@ -41,7 +41,7 @@ public static PhysicalPlanReader defaultPhysicalPlanReader( ScanResult scanResult = ClassPathScanner.fromPrescan(c); return new PhysicalPlanReader( c, scanResult, new LogicalPlanPersistence(c, scanResult), - CoordinationProtos.DrillbitEndpoint.getDefaultInstance(), + DrillbitEndpoint.getDefaultInstance(), storageRegistry); } public static PhysicalPlanReader defaultPhysicalPlanReader(DrillConfig c) { @@ -59,8 +59,14 @@ public static PhysicalPlanReader defaultPhysicalPlanReader( c.getConfig(), c.getClasspathScan(), c.getLpPersistence(), - CoordinationProtos.DrillbitEndpoint.getDefaultInstance(), + DrillbitEndpoint.getDefaultInstance(), storageRegistry); } + public static PhysicalPlanReader defaultPhysicalPlanReader(DrillConfig c, StoragePluginRegistry storageRegistry, + DrillbitEndpoint ep) { + ScanResult scanResult = ClassPathScanner.fromPrescan(c); + return new PhysicalPlanReader(c, scanResult, new LogicalPlanPersistence(c, scanResult), ep, storageRegistry); + } + } diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/planner/rm/TestMemoryCalculator.java b/exec/java-exec/src/test/java/org/apache/drill/exec/planner/rm/TestMemoryCalculator.java index fd3d454f78a..cd6b0a9a280 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/planner/rm/TestMemoryCalculator.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/planner/rm/TestMemoryCalculator.java @@ -22,11 +22,12 @@ import org.apache.drill.exec.ExecConstants; import org.apache.drill.exec.ops.QueryContext; import org.apache.drill.exec.planner.PhysicalPlanReader; +import org.apache.drill.exec.planner.fragment.DistributedQueueParallelizer; import org.apache.drill.exec.planner.fragment.Fragment; import org.apache.drill.exec.planner.fragment.PlanningSet; -import org.apache.drill.exec.planner.fragment.DistributedQueueParallelizer; import org.apache.drill.exec.planner.fragment.SimpleParallelizer; import org.apache.drill.exec.planner.fragment.Wrapper; +import org.apache.drill.exec.planner.fragment.common.DrillNode; import org.apache.drill.exec.pop.PopUnitTestBase; import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint; import org.apache.drill.exec.proto.UserBitShared; @@ -43,14 +44,16 @@ import org.apache.drill.test.ClusterFixture; import org.apache.drill.test.ClusterFixtureBuilder; import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; import org.junit.Test; import java.util.ArrayList; +import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.HashMap; import java.util.Set; import java.util.stream.Collectors; @@ -86,6 +89,9 @@ private static final DrillbitEndpoint newDrillbitEndpoint(String address, int po private static final QueryContext queryContext = new QueryContext(session, drillbitContext, UserBitShared.QueryId.getDefaultInstance()); + private static Map onlineEndpoints; + private Map resources; + @AfterClass public static void close() throws Exception { queryContext.close(); @@ -102,7 +108,7 @@ private QueryResourceManager mockResourceManager() throws QueueSelectionExceptio } private final Wrapper mockWrapper(Wrapper rootFragment, - Map resourceMap, + Map resourceMap, List endpoints, Map originalToMockWrapper ) { final Wrapper mockWrapper = mock(Wrapper.class); @@ -123,7 +129,7 @@ private final Wrapper mockWrapper(Wrapper rootFragment, } private final PlanningSet mockPlanningSet(PlanningSet planningSet, - Map resourceMap, + Map resourceMap, List endpoints) { Map wrapperToMockWrapper = new HashMap<>(); Wrapper rootFragment = mockWrapper( planningSet.getRootWrapper(), resourceMap, @@ -160,7 +166,7 @@ private String getPlanForQuery(String query, long outputBatchSize, return plan; } - private Map getEndpoints(int totalMinorFragments, + private static Map getEndpoints(int totalMinorFragments, Set notIn) { Map endpoints = new HashMap<>(); Iterator drillbits = Iterables.cycle(nodeList).iterator(); @@ -190,18 +196,26 @@ private Fragment getRootFragmentFromPlan(DrillbitContext context, } private PlanningSet preparePlanningSet(List activeEndpoints, long slice_target, - Map resources, String sql, + Map resources, String sql, SimpleParallelizer parallelizer) throws Exception { Fragment rootFragment = getRootFragmentFromPlan(drillbitContext, getPlanForQuery(sql, 10, slice_target)); return mockPlanningSet(parallelizer.prepareFragmentTree(rootFragment), resources, activeEndpoints); } + @BeforeClass + public static void setupForAllTests() { + onlineEndpoints = getEndpoints(2, new HashSet<>()); + } + + @Before + public void setupForEachTest() { + // Have to create separately for each test since it is updated my MemoryCalculator during merge + resources = onlineEndpoints.keySet().stream().collect(Collectors.toMap(x -> DrillNode.create(x), + x -> NodeResources.create())); + } + @Test public void TestSingleMajorFragmentWithProjectAndScan() throws Exception { - Map onlineEndpoints = getEndpoints(2, new HashSet<>()); - Map resources = onlineEndpoints.keySet().stream() - .collect(Collectors.toMap(x -> x, - x -> NodeResources.create())); String sql = "SELECT * from cp.`tpch/nation.parquet`"; SimpleParallelizer parallelizer = new DistributedQueueParallelizer(false, queryContext, mockResourceManager()); @@ -213,10 +227,6 @@ public void TestSingleMajorFragmentWithProjectAndScan() throws Exception { @Test public void TestSingleMajorFragmentWithGroupByProjectAndScan() throws Exception { - Map onlineEndpoints = getEndpoints(2, new HashSet<>()); - Map resources = onlineEndpoints.keySet().stream() - .collect(Collectors.toMap(x -> x, - x -> NodeResources.create())); String sql = "SELECT dept_id, count(*) from cp.`tpch/lineitem.parquet` group by dept_id"; SimpleParallelizer parallelizer = new DistributedQueueParallelizer(false, queryContext, mockResourceManager()); @@ -228,14 +238,11 @@ public void TestSingleMajorFragmentWithGroupByProjectAndScan() throws Exception @Test public void TestTwoMajorFragmentWithSortyProjectAndScan() throws Exception { - Map onlineEndpoints = getEndpoints(2, new HashSet<>()); - Map resources = onlineEndpoints.keySet().stream() - .collect(Collectors.toMap(x -> x, - x -> NodeResources.create())); String sql = "SELECT * from cp.`tpch/lineitem.parquet` order by dept_id"; SimpleParallelizer parallelizer = new DistributedQueueParallelizer(false, queryContext, mockResourceManager()); - PlanningSet planningSet = preparePlanningSet(new ArrayList<>(onlineEndpoints.keySet()), 2, resources, sql, parallelizer); + PlanningSet planningSet = preparePlanningSet(new ArrayList<>(onlineEndpoints.keySet()), 2, resources, sql, + parallelizer); parallelizer.adjustMemory(planningSet, createSet(planningSet.getRootWrapper()), onlineEndpoints); assertTrue("memory requirement is different", Iterables.all(resources.entrySet(), (e) -> e.getValue().getMemoryInBytes() == 481490)); } diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/pop/TestFragmentChecker.java b/exec/java-exec/src/test/java/org/apache/drill/exec/pop/TestFragmentChecker.java index 0c174d4a3be..67ab58eaec3 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/pop/TestFragmentChecker.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/pop/TestFragmentChecker.java @@ -17,9 +17,6 @@ */ package org.apache.drill.exec.pop; -import java.util.Map; -import java.util.HashMap; - import org.apache.drill.categories.PlannerTest; import org.apache.drill.exec.planner.PhysicalPlanReader; import org.apache.drill.exec.planner.PhysicalPlanReaderTestFactory; @@ -35,9 +32,11 @@ import org.apache.drill.exec.util.Utilities; import org.apache.drill.exec.work.QueryWorkUnit; import org.junit.Test; - import org.junit.experimental.categories.Category; +import java.util.HashMap; +import java.util.Map; + import static org.junit.Assert.assertEquals; @Category(PlannerTest.class) @@ -51,8 +50,6 @@ public void checkSimpleExchangePlan() throws Exception{ } private void print(String fragmentFile, int bitCount, int expectedFragmentCount) throws Exception { - PhysicalPlanReader ppr = PhysicalPlanReaderTestFactory.defaultPhysicalPlanReader(CONFIG); - Fragment fragmentRoot = getRootFragment(ppr, fragmentFile); SimpleParallelizer par = new DefaultParallelizer(true, 1000*1000, 5, 10, 1.2); Map endpoints = new HashMap<>(); DrillbitEndpoint localBit = null; @@ -65,6 +62,8 @@ private void print(String fragmentFile, int bitCount, int expectedFragmentCount) endpoints.put(b1, sb.append("Drillbit-").append(i).toString()); } + final PhysicalPlanReader ppr = PhysicalPlanReaderTestFactory.defaultPhysicalPlanReader(CONFIG, null, localBit); + Fragment fragmentRoot = getRootFragment(ppr, fragmentFile); final QueryContextInformation queryContextInfo = Utilities.createQueryContextInfo("dummySchemaName", "938ea2d9-7cb9-4baf-9414-a5a0b7777e8e"); QueryWorkUnit qwu = par.generateWorkUnit(new OptionList(), localBit, QueryId.getDefaultInstance(), endpoints, fragmentRoot, UserSession.Builder.newBuilder().withCredentials(UserBitShared.UserCredentials.newBuilder().setUserName("foo").build()).build(), diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/resourcemgr/RMBlobManagerTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/resourcemgr/RMBlobManagerTest.java index 89657e13e70..79d2c54e667 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/resourcemgr/RMBlobManagerTest.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/resourcemgr/RMBlobManagerTest.java @@ -21,9 +21,15 @@ import com.typesafe.config.ConfigFactory; import com.typesafe.config.ConfigValueFactory; import org.apache.drill.exec.ExecConstants; +import org.apache.drill.exec.proto.UserBitShared; import org.apache.drill.exec.resourcemgr.config.QueryQueueConfig; import org.apache.drill.exec.resourcemgr.config.QueryQueueConfigImpl; +import org.apache.drill.exec.resourcemgr.config.ResourcePoolTree; +import org.apache.drill.exec.resourcemgr.config.ResourcePoolTreeImpl; import org.apache.drill.exec.resourcemgr.rmblobmgr.RMConsistentBlobStoreManager; +import org.apache.drill.exec.resourcemgr.rmblobmgr.exception.LeaderChangeException; +import org.apache.drill.exec.resourcemgr.rmblobmgr.exception.RMBlobUpdateException; +import org.apache.drill.exec.resourcemgr.rmblobmgr.exception.ResourceUnavailableException; import org.apache.drill.exec.resourcemgr.rmblobmgr.rmblob.ClusterStateBlob; import org.apache.drill.exec.resourcemgr.rmblobmgr.rmblob.ForemanQueueUsageBlob; import org.apache.drill.exec.resourcemgr.rmblobmgr.rmblob.ForemanResourceUsage; @@ -40,11 +46,15 @@ import java.util.ArrayList; import java.util.HashMap; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.UUID; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; public class RMBlobManagerTest extends DrillTest { //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RMBlobManagerTest.class); @@ -55,117 +65,285 @@ public class RMBlobManagerTest extends DrillTest { private RMConsistentBlobStoreManager rmConsistentBlobStoreManager; + private final NodeResources nodeResourceShare = new NodeResources(65535, 10); + private ClusterStateBlob clusterStateBlob; private QueueLeadershipBlob queueLeadershipBlob; private ForemanQueueUsageBlob foremanQueueUsageBlob; - private final List leafQueues = new ArrayList<>(); + private ForemanResourceUsage foreman1RsrcUsage; + + private ForemanResourceUsage foreman2RsrcUsage; + + private ForemanResourceUsage foreman3RsrcUsage; + + private final Map leafQueues = new HashMap<>(); + + private final List drillUUID = new ArrayList<>(); + + private final Map blobsToSerialize = new HashMap<>(); + + private ClusterFixture cluster; @Before public void testSetup() throws Exception { - final Map queueConfigValues = new HashMap<>(); - queueConfigValues.put(QueryQueueConfigImpl.MAX_QUERY_MEMORY_PER_NODE_KEY, "8192K"); - - queueConfig = ConfigFactory.empty().withValue("queue", ConfigValueFactory.fromMap(queueConfigValues)); - - final QueryQueueConfig leafQueue1 = new QueryQueueConfigImpl(queueConfig.getConfig("queue"), "queue1", - null); - final QueryQueueConfig leafQueue2 = new QueryQueueConfigImpl(queueConfig.getConfig("queue"), "queue2", - null); - final QueryQueueConfig leafQueue3 = new QueryQueueConfigImpl(queueConfig.getConfig("queue"), "queue3", - null); - - leafQueues.add(leafQueue1); - leafQueues.add(leafQueue2); - leafQueues.add(leafQueue3); - - final List drillUUID = new ArrayList<>(); - drillUUID.add(UUID.randomUUID().toString()); - drillUUID.add(UUID.randomUUID().toString()); - drillUUID.add(UUID.randomUUID().toString()); - - final Map clusterStateValue = new HashMap<>(); - clusterStateValue.put(drillUUID.get(0), new NodeResources(65535, 10)); - clusterStateValue.put(drillUUID.get(1), new NodeResources(65535, 10)); - clusterStateValue.put(drillUUID.get(2), new NodeResources(65535, 10)); - - final Map queueLeadersValue = new HashMap<>(); - queueLeadersValue.put(leafQueue1.getQueueName(), drillUUID.get(0)); - queueLeadersValue.put(leafQueue2.getQueueName(), drillUUID.get(1)); - queueLeadersValue.put(leafQueue3.getQueueName(), drillUUID.get(2)); - - final Map foreman1Usage = new HashMap<>(); - foreman1Usage.put(drillUUID.get(1), new NodeResources(1000, 1)); - foreman1Usage.put(drillUUID.get(2), new NodeResources(2000, 1)); - - final Map foreman2Usage = new HashMap<>(); - foreman2Usage.put(drillUUID.get(0), new NodeResources(1000, 1)); - foreman2Usage.put(drillUUID.get(2), new NodeResources(2000, 1)); - - final Map foreman3Usage = new HashMap<>(); - foreman3Usage.put(drillUUID.get(0), new NodeResources(1000, 1)); - foreman3Usage.put(drillUUID.get(1), new NodeResources(2000, 1)); - - - final ForemanResourceUsage foreman1 = new ForemanResourceUsage(RMConsistentBlobStoreManager.RM_STATE_BLOB_VERSION, - foreman1Usage, 1); - final ForemanResourceUsage foreman2 = new ForemanResourceUsage(RMConsistentBlobStoreManager.RM_STATE_BLOB_VERSION, - foreman2Usage, 2); - final ForemanResourceUsage foreman3 = new ForemanResourceUsage(RMConsistentBlobStoreManager.RM_STATE_BLOB_VERSION, - foreman3Usage, 3); - - final Map formemanQueueUsageValues = new HashMap<>(); - formemanQueueUsageValues.put(drillUUID.get(0), foreman1); - formemanQueueUsageValues.put(drillUUID.get(1), foreman2); - formemanQueueUsageValues.put(drillUUID.get(2), foreman3); - - clusterStateBlob = new ClusterStateBlob(RMConsistentBlobStoreManager.RM_STATE_BLOB_VERSION, - clusterStateValue); - queueLeadershipBlob = new QueueLeadershipBlob(RMConsistentBlobStoreManager.RM_STATE_BLOB_VERSION, - queueLeadersValue); - foremanQueueUsageBlob = new ForemanQueueUsageBlob(RMConsistentBlobStoreManager.RM_STATE_BLOB_VERSION, - formemanQueueUsageValues); + final Map queueConfigValues = new HashMap<>(); + queueConfigValues.put(QueryQueueConfigImpl.MAX_QUERY_MEMORY_PER_NODE_KEY, "8192K"); + + queueConfig = ConfigFactory.empty().withValue("queue", ConfigValueFactory.fromMap(queueConfigValues)); + + final QueryQueueConfig leafQueue1 = new QueryQueueConfigImpl(queueConfig.getConfig("queue"), "queue1", + null); + final QueryQueueConfig leafQueue2 = new QueryQueueConfigImpl(queueConfig.getConfig("queue"), "queue2", + null); + final QueryQueueConfig leafQueue3 = new QueryQueueConfigImpl(queueConfig.getConfig("queue"), "queue3", + null); + + leafQueues.put("queue1", leafQueue1); + leafQueues.put("queue2", leafQueue2); + leafQueues.put("queue3", leafQueue3); + + drillUUID.add(UUID.randomUUID().toString()); + drillUUID.add(UUID.randomUUID().toString()); + drillUUID.add(UUID.randomUUID().toString()); + + final Map clusterStateValue = new HashMap<>(); + clusterStateValue.put(drillUUID.get(0), nodeResourceShare); + clusterStateValue.put(drillUUID.get(1), nodeResourceShare); + clusterStateValue.put(drillUUID.get(2), nodeResourceShare); + + final Map queueLeadersValue = new HashMap<>(); + queueLeadersValue.put(leafQueue1.getQueueName(), drillUUID.get(0)); + queueLeadersValue.put(leafQueue2.getQueueName(), drillUUID.get(1)); + queueLeadersValue.put(leafQueue3.getQueueName(), drillUUID.get(2)); + + final Map foreman1Usage = new HashMap<>(); + final NodeResources foreman1Resource = new NodeResources(1000, 1); + foreman1Usage.put(drillUUID.get(0), foreman1Resource); + foreman1Usage.put(drillUUID.get(1), foreman1Resource); + foreman1Usage.put(drillUUID.get(2), foreman1Resource); + + final Map foreman2Usage = new HashMap<>(); + final NodeResources foreman2Resource = new NodeResources(2000, 1); + foreman2Usage.put(drillUUID.get(0), foreman2Resource); + foreman2Usage.put(drillUUID.get(1), foreman2Resource); + foreman2Usage.put(drillUUID.get(2), foreman2Resource); + + final Map foreman3Usage = new HashMap<>(); + final NodeResources foreman3Resource = new NodeResources(3000, 1); + foreman3Usage.put(drillUUID.get(0), foreman3Resource); + foreman3Usage.put(drillUUID.get(1), foreman3Resource); + foreman3Usage.put(drillUUID.get(2), foreman3Resource); + + foreman1RsrcUsage = new ForemanResourceUsage(RMConsistentBlobStoreManager.RM_STATE_BLOB_VERSION, + foreman1Usage, 1); + foreman2RsrcUsage = new ForemanResourceUsage(RMConsistentBlobStoreManager.RM_STATE_BLOB_VERSION, + foreman2Usage, 2); + foreman3RsrcUsage = new ForemanResourceUsage(RMConsistentBlobStoreManager.RM_STATE_BLOB_VERSION, + foreman3Usage, 3); + + final Map formemanQueueUsageValues = new HashMap<>(); + formemanQueueUsageValues.put(drillUUID.get(0), foreman1RsrcUsage); + formemanQueueUsageValues.put(drillUUID.get(1), foreman2RsrcUsage); + formemanQueueUsageValues.put(drillUUID.get(2), foreman3RsrcUsage); + + clusterStateBlob = new ClusterStateBlob(RMConsistentBlobStoreManager.RM_STATE_BLOB_VERSION, clusterStateValue); + queueLeadershipBlob = new QueueLeadershipBlob(RMConsistentBlobStoreManager.RM_STATE_BLOB_VERSION, + queueLeadersValue); + foremanQueueUsageBlob = new ForemanQueueUsageBlob(RMConsistentBlobStoreManager.RM_STATE_BLOB_VERSION, + formemanQueueUsageValues); + + final ClusterFixtureBuilder fixtureBuilder = ClusterFixture.builder(dirTestWatcher) + .configProperty(ExecConstants.DRILL_PORT_HUNT, true) + .configProperty(ExecConstants.RM_ENABLED, false) + .setOptionDefault(ExecConstants.ENABLE_QUEUE.getOptionName(), false) + .withLocalZk(); + + // start the cluster + cluster = fixtureBuilder.build(); + + // prepare the blob cache + blobsToSerialize.put(ClusterStateBlob.NAME, clusterStateBlob); + blobsToSerialize.put(QueueLeadershipBlob.NAME, queueLeadershipBlob); + + for (QueryQueueConfig leafQueue : leafQueues.values()) { + String blobName = ForemanQueueUsageBlob.NAME + "_" + leafQueue.getQueueName(); + blobsToSerialize.put(blobName, foremanQueueUsageBlob); + } + + // initialize the blobs + final DrillbitContext context = cluster.drillbit().getContext(); + final ResourcePoolTree rmPoolTree = mock(ResourcePoolTreeImpl.class); + when(rmPoolTree.getAllLeafQueues()).thenReturn(leafQueues); + when(rmPoolTree.getRootPoolResources()).thenReturn(nodeResourceShare); + rmConsistentBlobStoreManager = new RMConsistentBlobStoreManager(context, rmPoolTree); + rmConsistentBlobStoreManager.writeAllRMBlobs(blobsToSerialize); + } + + private void verifyBlobs() { + // Again verify the updated blob value with initial value + Iterator> blobs = rmConsistentBlobStoreManager.readAllRMBlobs(); + while(blobs.hasNext()) { + final Map.Entry currentBlob = blobs.next(); + if (currentBlob.getKey().equals(ClusterStateBlob.NAME)) { + final ClusterStateBlob newStateBlob = (ClusterStateBlob) currentBlob.getValue(); + assertTrue(clusterStateBlob.equals(newStateBlob)); + } else if (currentBlob.getKey().equals(QueueLeadershipBlob.NAME)) { + assertTrue(queueLeadershipBlob.equals(currentBlob.getValue())); + } else { + assertTrue(foremanQueueUsageBlob.equals(currentBlob.getValue())); + } + } } @Test public void testRMStateBlobSerDe() throws Exception { - ClusterFixtureBuilder fixtureBuilder = ClusterFixture.builder(dirTestWatcher) - .configProperty(ExecConstants.DRILL_PORT_HUNT, true) - .withLocalZk(); + final Map serializedBlobs = rmConsistentBlobStoreManager.serializePassedInBlob(blobsToSerialize); + final Map deserializedBlobs = + rmConsistentBlobStoreManager.deserializeRMStateBlobs(serializedBlobs); - try (ClusterFixture cluster = fixtureBuilder.build()) { - final DrillbitContext context = cluster.drillbit().getContext(); - rmConsistentBlobStoreManager = new RMConsistentBlobStoreManager(context, leafQueues); - Map blobsToSerialize = new HashMap<>(); - blobsToSerialize.put(ClusterStateBlob.NAME, clusterStateBlob); - blobsToSerialize.put(QueueLeadershipBlob.NAME, queueLeadershipBlob); + for (Map.Entry blobEntry : deserializedBlobs.entrySet()) { + final RMStateBlob actualBlob = blobEntry.getValue(); + assertEquals(blobsToSerialize.get(blobEntry.getKey()), actualBlob); + } + } - for (QueryQueueConfig leafQueue : leafQueues) { - String blobName = ForemanQueueUsageBlob.NAME + "_" + leafQueue.getQueueName(); - blobsToSerialize.put(blobName, foremanQueueUsageBlob); + @Test + public void testSuccessfulReserveAndFree() throws Exception { + // Now let's reserve some resources for a query through reserve api + final Map resourceToReserve = new HashMap<>(); + resourceToReserve.put(drillUUID.get(0), new NodeResources(15535, 1)); + resourceToReserve.put(drillUUID.get(1), new NodeResources(15535, 1)); + + final String foremanUUID = drillUUID.get(1); + final UserBitShared.QueryId queryId = UserBitShared.QueryId.getDefaultInstance(); + rmConsistentBlobStoreManager.reserveResources(resourceToReserve, leafQueues.get("queue1"), + drillUUID.get(0), queryId.toString(), foremanUUID); + + // Verify the updated blob value with expected value + Iterator> blobs = rmConsistentBlobStoreManager.readAllRMBlobs(); + while(blobs.hasNext()) { + final Map.Entry currentBlob = blobs.next(); + if (currentBlob.getKey().equals(ClusterStateBlob.NAME)) { + final ClusterStateBlob newStateBlob = (ClusterStateBlob) currentBlob.getValue(); + final Map clusterState = newStateBlob.getClusterState(); + assertEquals( + nodeResourceShare.getMemoryInBytes() - resourceToReserve.get(drillUUID.get(0)).getMemoryInBytes(), + clusterState.get(drillUUID.get(0)).getMemoryInBytes()); + assertEquals( + nodeResourceShare.getMemoryInBytes() - resourceToReserve.get(drillUUID.get(1)).getMemoryInBytes(), + clusterState.get(drillUUID.get(1)).getMemoryInBytes()); + } else if (currentBlob.getKey().equals(ForemanQueueUsageBlob.NAME + "_queue1")) { + final ForemanQueueUsageBlob foremanUsage = (ForemanQueueUsageBlob) currentBlob.getValue(); + final ForemanResourceUsage queryForemanUsage = foremanUsage.getAllForemanInfo().get(foremanUUID); + assertEquals(foreman2RsrcUsage.getRunningCount() + 1, queryForemanUsage.getRunningCount()); + final Map otherDrillbitResourcesUsed = foreman2RsrcUsage.getForemanUsage(); + assertEquals(otherDrillbitResourcesUsed.get(drillUUID.get(0)).getMemoryInBytes() + + resourceToReserve.get(drillUUID.get(0)).getMemoryInBytes(), + queryForemanUsage.getForemanUsage().get(drillUUID.get(0)).getMemoryInBytes()); + assertEquals(otherDrillbitResourcesUsed.get(drillUUID.get(1)).getMemoryInBytes() + + resourceToReserve.get(drillUUID.get(1)).getMemoryInBytes(), + queryForemanUsage.getForemanUsage().get(drillUUID.get(1)).getMemoryInBytes()); } + } - final Map serializedBlobs = rmConsistentBlobStoreManager.serializePassedInBlob(blobsToSerialize); - final Map deserializedBlobs = rmConsistentBlobStoreManager.deserializeRMStateBlobs(serializedBlobs); + // release the resource back + rmConsistentBlobStoreManager.freeResources(resourceToReserve, leafQueues.get("queue1"), + drillUUID.get(0), queryId.toString(), foremanUUID); - for (Map.Entry blobEntry : deserializedBlobs.entrySet()) { - final RMStateBlob actualBlob = blobEntry.getValue(); - assertEquals(blobsToSerialize.get(blobEntry.getKey()), actualBlob); - } + // Again verify the updated blob value with initial value + verifyBlobs(); + } + + @Test (expected = RMBlobUpdateException.class) + public void testNonExistingNodeDuringReserve() throws Exception { + testNonExistingNodeCommon(false); + } + + @Test + public void testNonExistingNodeDuringFree() throws Exception { + testNonExistingNodeCommon(true); + } + + private void testNonExistingNodeCommon(boolean isFree) throws Exception { + // Now let's reserve some resources for a query through reserve api + final Map resourceToReserve = new HashMap<>(); + resourceToReserve.put(UUID.randomUUID().toString(), + new NodeResources(nodeResourceShare.getMemoryInBytes() + 1, 1)); + resourceToReserve.put(drillUUID.get(1), new NodeResources(nodeResourceShare.getMemoryInBytes(), 1)); + + final String foremanUUID = drillUUID.get(1); + final UserBitShared.QueryId queryId = UserBitShared.QueryId.getDefaultInstance(); + if (isFree) { + rmConsistentBlobStoreManager.freeResources(resourceToReserve, leafQueues.get("queue1"), drillUUID.get(0), + queryId.toString(), foremanUUID); + } else { + rmConsistentBlobStoreManager.reserveResources(resourceToReserve, leafQueues.get("queue1"), drillUUID.get(0), + queryId.toString(), foremanUUID); } } - public void testBlobManagerReserveApi() throws Exception { - ClusterFixtureBuilder fixtureBuilder = ClusterFixture.builder(dirTestWatcher) - .configProperty(ExecConstants.DRILL_PORT_HUNT, true) - .withLocalZk(); + @Test (expected = LeaderChangeException.class) + public void testLeaderChangeForQueueOnReserve() throws Exception { + testLeaderChangeCommon(false); + } + + @Test + public void testLeaderChangeForQueueOnFree() throws Exception { + testLeaderChangeCommon(true); + verifyBlobs(); + } - try (ClusterFixture cluster = fixtureBuilder.build()) { - DrillbitContext context = cluster.drillbit().getContext(); - final RMConsistentBlobStoreManager rmManager = new RMConsistentBlobStoreManager(context, leafQueues); + private void testLeaderChangeCommon(boolean isFree) throws Exception { + // First reserve some resources for a query through reserve api + final Map resourceToReserve = new HashMap<>(); + resourceToReserve.put(drillUUID.get(0), new NodeResources(4000, 1)); + resourceToReserve.put(drillUUID.get(1), new NodeResources(4000, 1)); + + final String foremanUUID = drillUUID.get(1); + final String leaderUUID = drillUUID.get(0); + final UserBitShared.QueryId queryId = UserBitShared.QueryId.getDefaultInstance(); + rmConsistentBlobStoreManager.reserveResources(resourceToReserve, leafQueues.get("queue1"), + (isFree) ? leaderUUID : UUID.randomUUID().toString(), queryId.toString(), foremanUUID); + + // now free up the query reserved resources + rmConsistentBlobStoreManager.freeResources(resourceToReserve, leafQueues.get("queue1"), + (isFree) ? UUID.randomUUID().toString() : leaderUUID, queryId.toString(), foremanUUID); + } + + @Test (expected = ResourceUnavailableException.class) + public void testReserveMoreThanAllowedForANode() throws Exception { + // Now let's reserve some resources for a query through reserve api + final Map resourceToReserve = new HashMap<>(); + resourceToReserve.put(drillUUID.get(0), new NodeResources(nodeResourceShare.getMemoryInBytes() + 1, + 1)); + resourceToReserve.put(drillUUID.get(1), new NodeResources(nodeResourceShare.getMemoryInBytes(), 1)); + + final String foremanUUID = drillUUID.get(1); + final UserBitShared.QueryId queryId = UserBitShared.QueryId.getDefaultInstance(); + rmConsistentBlobStoreManager.reserveResources(resourceToReserve, leafQueues.get("queue1"), + drillUUID.get(0), queryId.toString(), foremanUUID); + } + @Test (expected = RMBlobUpdateException.class) + public void testBlobAbsentBeforeUpdate() throws Exception { + try { + final Map resourceToReserve = new HashMap<>(); + resourceToReserve.put(drillUUID.get(0), new NodeResources(nodeResourceShare.getMemoryInBytes() + 1, + 1)); + resourceToReserve.put(drillUUID.get(1), new NodeResources(nodeResourceShare.getMemoryInBytes(), 1)); + + final String foremanUUID = drillUUID.get(1); + final UserBitShared.QueryId queryId = UserBitShared.QueryId.getDefaultInstance(); + + rmConsistentBlobStoreManager.deleteAllRMBlobs(new ArrayList<>(blobsToSerialize.keySet())); + rmConsistentBlobStoreManager.reserveResources(resourceToReserve, leafQueues.get("queue1"), drillUUID.get(0), + queryId.toString(), foremanUUID); + } finally { + // restore the blobs + rmConsistentBlobStoreManager.writeAllRMBlobs(blobsToSerialize); } } } diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/resourcemgr/TestDistributedQueryRM.java b/exec/java-exec/src/test/java/org/apache/drill/exec/resourcemgr/TestDistributedQueryRM.java new file mode 100644 index 00000000000..b19abf4013e --- /dev/null +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/resourcemgr/TestDistributedQueryRM.java @@ -0,0 +1,156 @@ +/* + * 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.drill.exec.resourcemgr; + +import org.apache.drill.exec.ExecConstants; +import org.apache.drill.exec.ops.QueryContext; +import org.apache.drill.exec.proto.UserBitShared; +import org.apache.drill.exec.server.DrillbitContext; +import org.apache.drill.exec.work.foreman.Foreman; +import org.apache.drill.exec.work.foreman.rm.DistributedResourceManager; +import org.apache.drill.exec.work.foreman.rm.QueryResourceManager; +import org.apache.drill.exec.work.foreman.rm.ResourceManager; +import org.apache.drill.shaded.guava.com.google.common.base.Preconditions; +import org.apache.drill.test.ClusterFixture; +import org.apache.drill.test.ClusterFixtureBuilder; +import org.apache.drill.test.ClusterTest; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; + +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class TestDistributedQueryRM extends ClusterTest { + + private static DistributedResourceManager drillRM; + + private static DrillbitContext context; + + private static Foreman mockForeman; + + private static final NodeResources testResources = new NodeResources(100, 1); + + private static final Map queryCosts = new HashMap<>(); + + private DistributedResourceManager.DistributedQueryRM queryRM; + + @BeforeClass + public static void setupTestSuite() throws Exception { + final ClusterFixtureBuilder fixtureBuilder = ClusterFixture.builder(dirTestWatcher) + .configProperty(ExecConstants.RM_ENABLED, true) + .setOptionDefault(ExecConstants.ENABLE_QUEUE.getOptionName(), false) + .configProperty(ExecConstants.DRILL_PORT_HUNT, true) + .withLocalZk(); + startCluster(fixtureBuilder); + context = cluster.drillbit().getContext(); + final ResourceManager rm = context.getResourceManager(); + Preconditions.checkState(rm instanceof DistributedResourceManager); + drillRM = (DistributedResourceManager) rm; + + mockForeman = mock(Foreman.class); + final QueryContext queryContext = mock(QueryContext.class); + when(mockForeman.getQueryContext()).thenReturn(queryContext); + when(queryContext.getQueryId()).thenReturn(UserBitShared.QueryId.getDefaultInstance()); + when(queryContext.getOnlineEndpointUUIDs()).thenReturn(context.getOnlineEndpointUUIDs()); + when(queryContext.getCurrentEndpoint()).thenReturn(context.getEndpoint()); + when(queryContext.getQueryUserName()).thenReturn(System.getProperty("user.name")); + final Collection keyUUIDs = context.getOnlineEndpointUUIDs().values(); + for (String keyID : keyUUIDs) { + queryCosts.put(keyID, testResources); + } + } + + @Before + public void testSetup() throws Exception { + queryRM = (DistributedResourceManager.DistributedQueryRM) drillRM.newQueryRM(mockForeman); + } + + @Test (expected = IllegalStateException.class) + public void testQueryRMReserve_NoQueue_NoCost() throws Exception { + queryRM.updateState(QueryResourceManager.QueryRMState.ENQUEUED); + // don't select queue and set cost for this query before reserving resources + queryRM.reserveResources(); + } + + @Test (expected = IllegalStateException.class) + public void testQueryRMReserve_NoCost() throws Exception { + // don't set cost for this query before reserving resources + queryRM.selectQueue(testResources); + queryRM.updateState(QueryResourceManager.QueryRMState.ENQUEUED); + queryRM.reserveResources(); + } + + @Test (expected = IllegalStateException.class) + public void testQueryRMReserve_BeforeAdmit() throws Exception { + // don't admit this query before reserving resources + queryRM.selectQueue(testResources); + queryRM.setCost(queryCosts); + queryRM.updateState(QueryResourceManager.QueryRMState.ENQUEUED); + queryRM.reserveResources(); + } + + @Test + public void testQueryRMReserveSuccess() throws Exception { + queryRM.selectQueue(testResources); + queryRM.setCost(queryCosts); + queryRM.updateState(QueryResourceManager.QueryRMState.ENQUEUED); + queryRM.updateState(QueryResourceManager.QueryRMState.ADMITTED); + queryRM.reserveResources(); + assertTrue(queryRM.getCurrentState() == QueryResourceManager.QueryRMState.RESERVED_RESOURCES); + queryRM.exit(); + assertTrue(queryRM.getCurrentState() == QueryResourceManager.QueryRMState.COMPLETED); + } + + @Test + public void testQueryRMExitInStartingState_QueryFailed() throws Exception { + when(mockForeman.getState()).thenReturn(UserBitShared.QueryResult.QueryState.FAILED); + queryRM.exit(); + assertTrue(queryRM.getCurrentState() == QueryResourceManager.QueryRMState.FAILED); + } + + @Test + public void testQueryRMExitInEnqueueState_QueryFailed() throws Exception { + when(mockForeman.getState()).thenReturn(UserBitShared.QueryResult.QueryState.FAILED); + queryRM.selectQueue(testResources); + queryRM.setCost(queryCosts); + queryRM.updateState(QueryResourceManager.QueryRMState.ENQUEUED); + queryRM.exit(); + assertTrue(queryRM.getCurrentState() == QueryResourceManager.QueryRMState.FAILED); + } + + @Test (expected = IllegalStateException.class) + public void testQueryRMExitInStartingState_QueryPreparing() throws Exception { + queryRM.exit(); + } + + @Test + public void testQueryRMAdmitted() throws Exception { + queryRM.selectQueue(testResources); + queryRM.setCost(queryCosts); + queryRM.admit(); + queryRM.updateState(QueryResourceManager.QueryRMState.ADMITTED); + queryRM.exit(); + assertTrue(queryRM.getCurrentState() == QueryResourceManager.QueryRMState.COMPLETED); + } +} diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/resourcemgr/TestE2EWithDistributedRM.java b/exec/java-exec/src/test/java/org/apache/drill/exec/resourcemgr/TestE2EWithDistributedRM.java new file mode 100644 index 00000000000..1479b55a947 --- /dev/null +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/resourcemgr/TestE2EWithDistributedRM.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.drill.exec.resourcemgr; + +import org.apache.drill.exec.ExecConstants; +import org.apache.drill.test.ClusterFixture; +import org.apache.drill.test.ClusterFixtureBuilder; +import org.apache.drill.test.ClusterTest; +import org.junit.BeforeClass; +import org.junit.Test; + +public class TestE2EWithDistributedRM extends ClusterTest { + //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestE2EWithDistributedRM.class); + private String query; + + @BeforeClass + public static void setupTestSuite() throws Exception { + final ClusterFixtureBuilder fixtureBuilder = ClusterFixture.builder(dirTestWatcher) + .configProperty(ExecConstants.RM_ENABLED, true) + .configProperty(ExecConstants.DRILL_PORT_HUNT, true) + .setOptionDefault(ExecConstants.ENABLE_QUEUE.getOptionName(), false) + .withLocalZk(); + startCluster(fixtureBuilder); + } + + @Test + public void testSystemTableQuery() throws Exception { + query = "SELECT * FROM sys.drillbits;"; + runAndLog(query); + } + + @Test + public void testNonBufferedOperatorQuery() throws Exception { + query = "SELECT * FROM cp.`employee.json` WHERE employee_id < 40 LIMIT 20"; + runAndLog(query); + } + + @Test + public void testBufferedOperatorQuery() throws Exception { + query = "SELECT * FROM cp.`employee.json` WHERE employee_id < 40 ORDER BY first_name"; + runAndLog(query); + } +} diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/resourcemgr/TestRMConfigLoad.java b/exec/java-exec/src/test/java/org/apache/drill/exec/resourcemgr/TestRMConfigLoad.java index 16f7f64f075..4b83a5f4321 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/resourcemgr/TestRMConfigLoad.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/resourcemgr/TestRMConfigLoad.java @@ -25,7 +25,9 @@ import org.apache.drill.exec.resourcemgr.config.selectors.AclSelector; import org.apache.drill.exec.work.foreman.rm.DefaultResourceManager; import org.apache.drill.exec.work.foreman.rm.DistributedResourceManager; +import org.apache.drill.exec.work.foreman.rm.DynamicResourceManager; import org.apache.drill.exec.work.foreman.rm.ResourceManager; +import org.apache.drill.exec.work.foreman.rm.ThrottledResourceManager; import org.apache.drill.test.BaseDirTestWatcher; import org.apache.drill.test.ClusterFixture; import org.apache.drill.test.ClusterFixtureBuilder; @@ -50,9 +52,10 @@ public void testDefaultRMConfig() throws Exception { ClusterFixtureBuilder fixtureBuilder = ClusterFixture.builder(dirTestWatcher) .configProperty(ExecConstants.RM_ENABLED, true) .configProperty(ExecConstants.DRILL_PORT_HUNT, true) + .setOptionDefault(ExecConstants.ENABLE_QUEUE.getOptionName(), false) .withLocalZk(); - try (ClusterFixture cluster = fixtureBuilder.build()) { + try(ClusterFixture cluster = fixtureBuilder.build()) { ResourceManager resourceManager = cluster.drillbit().getContext().getResourceManager(); assertTrue(resourceManager instanceof DistributedResourceManager); @@ -83,7 +86,8 @@ public void testDefaultRMConfig() throws Exception { @Test public void testDefaultRMWithLocalCoordinatorAndRMEnabled() throws Exception { ClusterFixtureBuilder fixtureBuilder = ClusterFixture.builder(dirTestWatcher) - .configProperty(ExecConstants.RM_ENABLED, true); + .configProperty(ExecConstants.RM_ENABLED, true) + .setOptionDefault(ExecConstants.ENABLE_QUEUE.getOptionName(), false); try (ClusterFixture cluster = fixtureBuilder.build()) { ResourceManager resourceManager = cluster.drillbit().getContext().getResourceManager(); @@ -94,7 +98,8 @@ public void testDefaultRMWithLocalCoordinatorAndRMEnabled() throws Exception { @Test public void testDefaultRMWithLocalCoordinatorAndRMDisabled() throws Exception { ClusterFixtureBuilder fixtureBuilder = ClusterFixture.builder(dirTestWatcher) - .configProperty(ExecConstants.RM_ENABLED, false); + .configProperty(ExecConstants.RM_ENABLED, false) + .setOptionDefault(ExecConstants.ENABLE_QUEUE.getOptionName(), false); try (ClusterFixture cluster = fixtureBuilder.build()) { ResourceManager resourceManager = cluster.drillbit().getContext().getResourceManager(); @@ -107,11 +112,28 @@ public void testDefaultRMOnlyRMDisabled() throws Exception { ClusterFixtureBuilder fixtureBuilder = ClusterFixture.builder(dirTestWatcher) .configProperty(ExecConstants.RM_ENABLED, false) .configProperty(ExecConstants.DRILL_PORT_HUNT, true) + .setOptionDefault(ExecConstants.ENABLE_QUEUE.getOptionName(), false) .withLocalZk(); try (ClusterFixture cluster = fixtureBuilder.build()) { ResourceManager resourceManager = cluster.drillbit().getContext().getResourceManager(); - assertTrue(resourceManager instanceof DefaultResourceManager); + assertTrue(resourceManager instanceof DynamicResourceManager); + assertTrue(((DynamicResourceManager) resourceManager).activeRM() instanceof DefaultResourceManager); + } + } + + @Test + public void testThrottleRMOnlyRMDisabled() throws Exception { + ClusterFixtureBuilder fixtureBuilder = ClusterFixture.builder(dirTestWatcher) + .configProperty(ExecConstants.RM_ENABLED, false) + .configProperty(ExecConstants.DRILL_PORT_HUNT, true) + .setOptionDefault(ExecConstants.ENABLE_QUEUE.getOptionName(), true) + .withLocalZk(); + + try (ClusterFixture cluster = fixtureBuilder.build()) { + ResourceManager resourceManager = cluster.drillbit().getContext().getResourceManager(); + assertTrue(resourceManager instanceof DynamicResourceManager); + assertTrue(((DynamicResourceManager) resourceManager).activeRM() instanceof ThrottledResourceManager); } } } diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/ExampleTest.java b/exec/java-exec/src/test/java/org/apache/drill/test/ExampleTest.java index afa1a87469b..0ed27f70cfd 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/test/ExampleTest.java +++ b/exec/java-exec/src/test/java/org/apache/drill/test/ExampleTest.java @@ -17,14 +17,7 @@ */ package org.apache.drill.test; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; - -import java.io.File; -import java.io.FileWriter; -import java.io.IOException; -import java.io.PrintWriter; - +import ch.qos.logback.classic.Level; import org.apache.drill.common.types.TypeProtos; import org.apache.drill.common.types.Types; import org.apache.drill.exec.ExecConstants; @@ -42,7 +35,13 @@ import org.junit.Rule; import org.junit.Test; -import ch.qos.logback.classic.Level; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.io.PrintWriter; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; /** * Example test case using the Drill cluster fixture. Your test case diff --git a/exec/memory/base/src/main/java/org/apache/drill/exec/memory/Accountant.java b/exec/memory/base/src/main/java/org/apache/drill/exec/memory/Accountant.java index cf55f5f51cc..f43e4a03030 100644 --- a/exec/memory/base/src/main/java/org/apache/drill/exec/memory/Accountant.java +++ b/exec/memory/base/src/main/java/org/apache/drill/exec/memory/Accountant.java @@ -17,16 +17,14 @@ */ package org.apache.drill.exec.memory; -import java.util.concurrent.atomic.AtomicLong; - -import javax.annotation.concurrent.ThreadSafe; - import org.apache.drill.exec.exception.OutOfMemoryException; import org.apache.drill.exec.util.AssertionUtil; - import org.apache.drill.shaded.guava.com.google.common.annotations.VisibleForTesting; import org.apache.drill.shaded.guava.com.google.common.base.Preconditions; +import javax.annotation.concurrent.ThreadSafe; +import java.util.concurrent.atomic.AtomicLong; + /** * Provides a concurrent way to manage account for memory usage without locking. Used as basis for Allocators. All * operations are threadsafe (except for close). @@ -111,8 +109,8 @@ public class Accountant implements AutoCloseable { public Accountant(Accountant parent, long reservation, long maxAllocation) { Preconditions.checkArgument(reservation >= 0, "The initial reservation size must be non-negative."); Preconditions.checkArgument(maxAllocation >= 0, "The maximum allocation limit must be non-negative."); - Preconditions.checkArgument(reservation <= maxAllocation, - "The initial reservation size must be <= the maximum allocation."); + Preconditions.checkArgument(reservation <= maxAllocation, String.format("The initial reservation size %d " + + "must be <= the maximum allocation %d.", reservation, maxAllocation)); Preconditions.checkArgument(reservation == 0 || parent != null, "The root accountant can't reserve memory."); this.parent = parent;