From 097122eb9c39a46a00a5b36117014cea0a3bd34c Mon Sep 17 00:00:00 2001 From: Sorabh Hamirwasia Date: Thu, 18 Apr 2019 14:50:38 -0700 Subject: [PATCH 1/7] DRILL-7164: KafkaFilterPushdownTest is sometimes failing to pattern match correctly --- .../store/kafka/KafkaFilterPushdownTest.java | 96 +++++++++++++------ .../java/org/apache/drill/PlanTestBase.java | 48 ++++++---- 2 files changed, 101 insertions(+), 43 deletions(-) diff --git a/contrib/storage-kafka/src/test/java/org/apache/drill/exec/store/kafka/KafkaFilterPushdownTest.java b/contrib/storage-kafka/src/test/java/org/apache/drill/exec/store/kafka/KafkaFilterPushdownTest.java index 2a5cdddf83c..52ca17189da 100644 --- a/contrib/storage-kafka/src/test/java/org/apache/drill/exec/store/kafka/KafkaFilterPushdownTest.java +++ b/contrib/storage-kafka/src/test/java/org/apache/drill/exec/store/kafka/KafkaFilterPushdownTest.java @@ -17,10 +17,10 @@ */ package org.apache.drill.exec.store.kafka; +import org.apache.drill.PlanTestBase; import org.apache.drill.categories.KafkaStorageTest; import org.apache.drill.categories.SlowTest; import org.apache.kafka.common.serialization.StringSerializer; - import org.junit.BeforeClass; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -32,10 +32,8 @@ @Category({KafkaStorageTest.class, SlowTest.class}) public class KafkaFilterPushdownTest extends KafkaTestBase { private static final int NUM_PARTITIONS = 5; - private static final String expectedSubStr = " \"kafkaScanSpec\" : {\n" + - " \"topicName\" : \"drill-pushdown-topic\"\n" + - " },\n" + - " \"cost\""; + private static final String expectedPattern = "kafkaScanSpec.*\\n.*\"topicName\" : \"drill-pushdown-topic\"\\n(" + + ".*\\n)?(.*\\n)?(.*\\n)?.*cost\"(.*\\n)(.*\\n).*outputRowCount\" : (%s.0)"; @BeforeClass public static void setup() throws Exception { @@ -63,7 +61,9 @@ public void testPushdownOnOffset() throws Exception { TestQueryConstants.JSON_PUSHDOWN_TOPIC, predicate1, predicate2); runKafkaSQLVerifyCount(queryString, expectedRowCount); - testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowCount)); + PlanTestBase.testPlanMatchingPatterns(queryString, JSON_FORMAT, + new String[] {String.format(expectedPattern, expectedRowCount)}, + new String[]{}); } /** @@ -79,7 +79,9 @@ public void testPushdownOnPartition() throws Exception { TestQueryConstants.JSON_PUSHDOWN_TOPIC, predicate); runKafkaSQLVerifyCount(queryString, expectedRowCount); - testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowCount)); + PlanTestBase.testPlanMatchingPatterns(queryString, JSON_FORMAT, + new String[] {String.format(expectedPattern, expectedRowCount)}, + new String[]{}); } /** @@ -95,7 +97,9 @@ public void testPushdownOnTimestamp() throws Exception { TestQueryConstants.JSON_PUSHDOWN_TOPIC, predicate); runKafkaSQLVerifyCount(queryString,expectedRowCount); - testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowCount)); + PlanTestBase.testPlanMatchingPatterns(queryString, JSON_FORMAT, + new String[] {String.format(expectedPattern, expectedRowCount)}, + new String[]{}); } /** @@ -112,7 +116,9 @@ public void testPushdownUnorderedTimestamp() throws Exception { TestQueryConstants.JSON_PUSHDOWN_TOPIC, predicate); runKafkaSQLVerifyCount(queryString,expectedRowCount); - testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowInPlan)); + PlanTestBase.testPlanMatchingPatterns(queryString, JSON_FORMAT, + new String[] {String.format(expectedPattern, expectedRowInPlan)}, + new String[]{}); } /** @@ -128,7 +134,9 @@ public void testPushdownWhenTimestampDoesNotExist() throws Exception { TestQueryConstants.JSON_PUSHDOWN_TOPIC, predicate); runKafkaSQLVerifyCount(queryString,expectedRowCount); - testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowCount)); + PlanTestBase.testPlanMatchingPatterns(queryString, JSON_FORMAT, + new String[] {String.format(expectedPattern, expectedRowCount)}, + new String[]{}); } /** @@ -144,7 +152,9 @@ public void testPushdownWhenPartitionDoesNotExist() throws Exception { TestQueryConstants.JSON_PUSHDOWN_TOPIC, predicate); runKafkaSQLVerifyCount(queryString,expectedRowCount); - testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowCount)); + PlanTestBase.testPlanMatchingPatterns(queryString, JSON_FORMAT, + new String[] {String.format(expectedPattern, expectedRowCount)}, + new String[]{}); } /** @@ -161,7 +171,9 @@ public void testPushdownForEmptyScanSpec() throws Exception { TestQueryConstants.JSON_PUSHDOWN_TOPIC, predicate1, predicate2); runKafkaSQLVerifyCount(queryString,expectedRowCount); - testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowCount)); + PlanTestBase.testPlanMatchingPatterns(queryString, JSON_FORMAT, + new String[] {String.format(expectedPattern, expectedRowCount)}, + new String[]{}); } /** @@ -178,42 +190,54 @@ public void testPushdownOffsetNoRecordsReturnedWithBoundaryConditions() throws E TestQueryConstants.JSON_PUSHDOWN_TOPIC, "kafkaMsgOffset = 10"); runKafkaSQLVerifyCount(queryString,expectedRowCount); - testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowCount)); + PlanTestBase.testPlanMatchingPatterns(queryString, JSON_FORMAT, + new String[] {String.format(expectedPattern, expectedRowCount)}, + new String[]{}); //"equal" such that value < startOffset queryString = String.format(TestQueryConstants.QUERY_TEMPLATE_BASIC, TestQueryConstants.JSON_PUSHDOWN_TOPIC, "kafkaMsgOffset = -1"); runKafkaSQLVerifyCount(queryString,expectedRowCount); - testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowCount)); + PlanTestBase.testPlanMatchingPatterns(queryString, JSON_FORMAT, + new String[] {String.format(expectedPattern, expectedRowCount)}, + new String[]{}); //"greater_than" such that value = endOffset-1 queryString = String.format(TestQueryConstants.QUERY_TEMPLATE_BASIC, TestQueryConstants.JSON_PUSHDOWN_TOPIC, "kafkaMsgOffset > 9"); runKafkaSQLVerifyCount(queryString,expectedRowCount); - testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowCount)); + PlanTestBase.testPlanMatchingPatterns(queryString, JSON_FORMAT, + new String[] {String.format(expectedPattern, expectedRowCount)}, + new String[]{}); //"greater_than_or_equal" such that value = endOffset queryString = String.format(TestQueryConstants.QUERY_TEMPLATE_BASIC, TestQueryConstants.JSON_PUSHDOWN_TOPIC, "kafkaMsgOffset >= 10"); runKafkaSQLVerifyCount(queryString,expectedRowCount); - testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowCount)); + PlanTestBase.testPlanMatchingPatterns(queryString, JSON_FORMAT, + new String[] {String.format(expectedPattern, expectedRowCount)}, + new String[]{}); //"less_than" such that value = startOffset queryString = String.format(TestQueryConstants.QUERY_TEMPLATE_BASIC, TestQueryConstants.JSON_PUSHDOWN_TOPIC, "kafkaMsgOffset < 0"); runKafkaSQLVerifyCount(queryString,expectedRowCount); - testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowCount)); + PlanTestBase.testPlanMatchingPatterns(queryString, JSON_FORMAT, + new String[] {String.format(expectedPattern, expectedRowCount)}, + new String[]{}); //"less_than_or_equal" such that value < startOffset queryString = String.format(TestQueryConstants.QUERY_TEMPLATE_BASIC, TestQueryConstants.JSON_PUSHDOWN_TOPIC, "kafkaMsgOffset <= -1"); runKafkaSQLVerifyCount(queryString,expectedRowCount); - testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowCount)); + PlanTestBase.testPlanMatchingPatterns(queryString, JSON_FORMAT, + new String[] {String.format(expectedPattern, expectedRowCount)}, + new String[]{}); } /** @@ -230,21 +254,27 @@ public void testPushdownOffsetOneRecordReturnedWithBoundaryConditions() throws E TestQueryConstants.JSON_PUSHDOWN_TOPIC, "kafkaMsgOffset = 9"); runKafkaSQLVerifyCount(queryString, expectedRowCount); - testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowCount)); + PlanTestBase.testPlanMatchingPatterns(queryString, JSON_FORMAT, + new String[] {String.format(expectedPattern, expectedRowCount)}, + new String[]{}); //"greater_than" such that value = endOffset-2 queryString = String.format(TestQueryConstants.QUERY_TEMPLATE_BASIC, TestQueryConstants.JSON_PUSHDOWN_TOPIC, "kafkaMsgOffset > 8"); runKafkaSQLVerifyCount(queryString,expectedRowCount); - testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowCount)); + PlanTestBase.testPlanMatchingPatterns(queryString, JSON_FORMAT, + new String[] {String.format(expectedPattern, expectedRowCount)}, + new String[]{}); //"greater_than_or_equal" such that value = endOffset-1 queryString = String.format(TestQueryConstants.QUERY_TEMPLATE_BASIC, TestQueryConstants.JSON_PUSHDOWN_TOPIC, "kafkaMsgOffset >= 9"); runKafkaSQLVerifyCount(queryString,expectedRowCount); - testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowCount)); + PlanTestBase.testPlanMatchingPatterns(queryString, JSON_FORMAT, + new String[] {String.format(expectedPattern, expectedRowCount)}, + new String[]{}); } /** @@ -262,7 +292,9 @@ public void testPushdownWithOr() throws Exception { TestQueryConstants.JSON_PUSHDOWN_TOPIC, predicate1, predicate2); runKafkaSQLVerifyCount(queryString,expectedRowCount); - testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowCount)); + PlanTestBase.testPlanMatchingPatterns(queryString, JSON_FORMAT, + new String[] {String.format(expectedPattern, expectedRowCount)}, + new String[]{}); } /** @@ -280,7 +312,9 @@ public void testPushdownWithOr1() throws Exception { TestQueryConstants.JSON_PUSHDOWN_TOPIC, predicate1, predicate2); runKafkaSQLVerifyCount(queryString,expectedRowCount); - testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowInPlan)); + PlanTestBase.testPlanMatchingPatterns(queryString, JSON_FORMAT, + new String[] {String.format(expectedPattern, expectedRowInPlan)}, + new String[]{}); } /** @@ -299,7 +333,9 @@ public void testPushdownWithAndOrCombo() throws Exception { TestQueryConstants.JSON_PUSHDOWN_TOPIC, predicate1, predicate2, predicate3); runKafkaSQLVerifyCount(queryString,expectedRowCount); - testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowCount)); + PlanTestBase.testPlanMatchingPatterns(queryString, JSON_FORMAT, + new String[] {String.format(expectedPattern, expectedRowCount)}, + new String[]{}); } /** @@ -319,7 +355,9 @@ public void testPushdownWithAndOrCombo2() throws Exception { TestQueryConstants.JSON_PUSHDOWN_TOPIC, predicate1, predicate2, predicate3, predicate4); runKafkaSQLVerifyCount(queryString,expectedRowCount); - testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowCountInPlan)); + PlanTestBase.testPlanMatchingPatterns(queryString, JSON_FORMAT, + new String[] {String.format(expectedPattern, expectedRowCountInPlan)}, + new String[]{}); } /** @@ -338,7 +376,9 @@ public void testPushdownTimestampWithNonMetaField() throws Exception { TestQueryConstants.JSON_PUSHDOWN_TOPIC, predicate1, predicate2); runKafkaSQLVerifyCount(queryString,expectedRowCount); - testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowCountInPlan)); + PlanTestBase.testPlanMatchingPatterns(queryString, JSON_FORMAT, + new String[] {String.format(expectedPattern, expectedRowCountInPlan)}, + new String[]{}); } /** @@ -358,7 +398,9 @@ public void testNoPushdownOfOffsetWithNonMetadataField() throws Exception { TestQueryConstants.JSON_PUSHDOWN_TOPIC, predicate1, predicate2, predicate3); runKafkaSQLVerifyCount(queryString,expectedRowCount); - testPhysicalPlan(queryString, String.format(expectedSubStr, expectedRowCountInPlan)); + PlanTestBase.testPlanMatchingPatterns(queryString, JSON_FORMAT, + new String[] {String.format(expectedPattern, expectedRowCountInPlan)}, + new String[]{}); } } diff --git a/exec/java-exec/src/test/java/org/apache/drill/PlanTestBase.java b/exec/java-exec/src/test/java/org/apache/drill/PlanTestBase.java index fae593f28fc..a7772764e57 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/PlanTestBase.java +++ b/exec/java-exec/src/test/java/org/apache/drill/PlanTestBase.java @@ -17,18 +17,8 @@ */ package org.apache.drill; -import java.nio.file.Paths; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -import java.io.File; -import java.io.IOException; -import java.nio.file.Files; -import java.util.List; -import java.util.Stack; -import java.util.regex.Matcher; -import java.util.regex.Pattern; - +import org.apache.calcite.sql.SqlExplain.Depth; +import org.apache.calcite.sql.SqlExplainLevel; import org.apache.commons.io.FileUtils; import org.apache.drill.common.expression.SchemaPath; import org.apache.drill.exec.record.RecordBatchLoader; @@ -37,13 +27,23 @@ import org.apache.drill.exec.store.parquet.metadata.Metadata; import org.apache.drill.exec.vector.NullableVarCharVector; import org.apache.drill.exec.vector.ValueVector; -import org.apache.calcite.sql.SqlExplain.Depth; -import org.apache.calcite.sql.SqlExplainLevel; - +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.test.BaseTestQuery; import org.apache.drill.test.QueryTestUtil; +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.List; +import java.util.Stack; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + public class PlanTestBase extends BaseTestQuery { static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PlanTestBase.class); @@ -92,7 +92,23 @@ public static void testPlanMatchingPatterns(String query, String[] expectedPatte public static void testPlanMatchingPatterns(String query, Pattern[] expectedPatterns, Pattern[] excludedPatterns) throws Exception { - final String plan = getPlanInString("EXPLAIN PLAN for " + QueryTestUtil.normalizeQuery(query), OPTIQ_FORMAT); + testPlanMatchingPatterns(query, OPTIQ_FORMAT, expectedPatterns, excludedPatterns); + } + + public static void testPlanMatchingPatterns(String query, String planFormat, + String[] expectedPatterns, String[] excludedPatterns) + throws Exception { + Preconditions.checkArgument((planFormat.equals(OPTIQ_FORMAT) || planFormat.equals(JSON_FORMAT)), "Unsupported " + + "plan format %s is provided for explain plan query. Supported formats are: %s, %s", planFormat, OPTIQ_FORMAT, + JSON_FORMAT); + testPlanMatchingPatterns(query, planFormat, stringsToPatterns(expectedPatterns), + stringsToPatterns(excludedPatterns)); + } + + private static void testPlanMatchingPatterns(String query, String planFormat, + Pattern[] expectedPatterns, Pattern[] excludedPatterns) + throws Exception { + final String plan = getPlanInString("EXPLAIN PLAN for " + QueryTestUtil.normalizeQuery(query), planFormat); // Check and make sure all expected patterns are in the plan if (expectedPatterns != null) { From 7e71cd03b4fb1bb6ca5132e9cffcf56e418b4cb3 Mon Sep 17 00:00:00 2001 From: HanumathRao Date: Thu, 14 Mar 2019 10:58:37 -0700 Subject: [PATCH 2/7] DRILL-7193: Integration changes of the Distributed RM queue configuration with Simple Parallelizer. Refactor existing ZK based queue to accommodate new Distributed queue for RM. Refactor and rename the existing memory allocation utilities to ZKQueueMemoryAllocationUtilities and DefaultMemoryAllocationUtilities. Parallelizer code is changed to accommodate the memory adjustment for the operators during parallelization phase. With this change, there are 3 different implementation of SimpleParallelizer; they are ZKQueueParallelizer, DistributedQueueParallelizer and DefaultParallelizer which will be used by ZK based RM, Distributed RM and Non RM configuration. --- ...llelizer.java => DefaultParallelizer.java} | 12 +- ...java => DistributedQueueParallelizer.java} | 4 +- .../drill/exec/planner/fragment/Fragment.java | 1 + .../planner/fragment/ZKQueueParallelizer.java | 121 +++++++++ .../contrib/SplittingParallelizer.java | 4 +- .../DefaultMemoryAllocationUtilities.java} | 6 +- .../ZKQueueMemoryAllocationUtilities.java | 256 ++++++++++++++++++ .../drill/exec/work/foreman/Foreman.java | 2 - .../foreman/rm/DefaultResourceManager.java | 53 +--- .../rm/DistributedResourceManager.java | 81 +++++- .../foreman/rm/DynamicResourceManager.java | 7 - .../foreman/rm/QueryResourceAllocator.java | 56 ---- .../work/foreman/rm/QueryResourceManager.java | 8 +- .../exec/work/foreman/rm/ResourceManager.java | 11 - .../foreman/rm/ThrottledResourceManager.java | 253 ++--------------- .../drill/exec/work/user/PlanSplitter.java | 8 +- .../exec/physical/impl/TestLocalExchange.java | 4 +- .../partitionsender/TestPartitionSender.java | 4 +- .../exec/planner/rm/TestMemoryCalculator.java | 8 +- .../drill/exec/pop/TestFragmentChecker.java | 4 +- .../drill/exec/util/TestQueryMemoryAlloc.java | 23 +- 21 files changed, 522 insertions(+), 404 deletions(-) rename exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/{DefaultQueryParallelizer.java => DefaultParallelizer.java} (81%) rename exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/{QueueQueryParallelizer.java => DistributedQueueParallelizer.java} (98%) create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/ZKQueueParallelizer.java rename exec/java-exec/src/main/java/org/apache/drill/exec/util/{MemoryAllocationUtilities.java => memory/DefaultMemoryAllocationUtilities.java} (97%) create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/util/memory/ZKQueueMemoryAllocationUtilities.java delete mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/QueryResourceAllocator.java diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/DefaultQueryParallelizer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/DefaultParallelizer.java similarity index 81% rename from exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/DefaultQueryParallelizer.java rename to exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/DefaultParallelizer.java index 400648f3482..b298fa793a5 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/DefaultQueryParallelizer.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/DefaultParallelizer.java @@ -20,7 +20,7 @@ import org.apache.drill.exec.ops.QueryContext; import org.apache.drill.exec.physical.base.PhysicalOperator; import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint; -import org.apache.drill.exec.util.MemoryAllocationUtilities; +import org.apache.drill.exec.util.memory.DefaultMemoryAllocationUtilities; import java.util.Collection; import java.util.List; @@ -32,18 +32,18 @@ * The memory computation of the operators is based on the earlier logic to assign memory for the buffered * operators. */ -public class DefaultQueryParallelizer extends SimpleParallelizer { +public class DefaultParallelizer extends SimpleParallelizer { private final boolean planHasMemory; private final QueryContext queryContext; - public DefaultQueryParallelizer(boolean memoryAvailableInPlan, QueryContext queryContext) { + public DefaultParallelizer(boolean memoryAvailableInPlan, QueryContext queryContext) { super(queryContext); this.planHasMemory = memoryAvailableInPlan; this.queryContext = queryContext; } - public DefaultQueryParallelizer(boolean memoryPlanning, long parallelizationThreshold, int maxWidthPerNode, - int maxGlobalWidth, double affinityFactor) { + public DefaultParallelizer(boolean memoryPlanning, long parallelizationThreshold, int maxWidthPerNode, + int maxGlobalWidth, double affinityFactor) { super(parallelizationThreshold, maxWidthPerNode, maxGlobalWidth, affinityFactor); this.planHasMemory = memoryPlanning; this.queryContext = null; @@ -56,7 +56,7 @@ public void adjustMemory(PlanningSet planningSet, Set roots, return; } List bufferedOpers = planningSet.getRootWrapper().getNode().getBufferedOperators(queryContext); - MemoryAllocationUtilities.setupBufferedOpsMemoryAllocations(planHasMemory, bufferedOpers, queryContext); + DefaultMemoryAllocationUtilities.setupBufferedOpsMemoryAllocations(planHasMemory, bufferedOpers, queryContext); } @Override diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/QueueQueryParallelizer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/DistributedQueueParallelizer.java similarity index 98% rename from exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/QueueQueryParallelizer.java rename to exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/DistributedQueueParallelizer.java index 5cd4a09e420..009c750452c 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/QueueQueryParallelizer.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/DistributedQueueParallelizer.java @@ -40,12 +40,12 @@ * cluster state. However, the memory assignment for each operator, minor fragment and major * fragment is based on the cluster state and provided queue configuration. */ -public class QueueQueryParallelizer extends SimpleParallelizer { +public class DistributedQueueParallelizer extends SimpleParallelizer { private final boolean planHasMemory; private final QueryContext queryContext; private final Map> operators; - public QueueQueryParallelizer(boolean memoryPlanning, QueryContext queryContext) { + public DistributedQueueParallelizer(boolean memoryPlanning, QueryContext queryContext) { super(queryContext); this.planHasMemory = memoryPlanning; this.queryContext = queryContext; diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Fragment.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Fragment.java index 35e5095f0ea..0212e088211 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Fragment.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Fragment.java @@ -25,6 +25,7 @@ import org.apache.drill.exec.physical.base.AbstractPhysicalVisitor; import org.apache.drill.exec.physical.base.Exchange; import org.apache.drill.exec.physical.base.PhysicalOperator; +import org.apache.drill.exec.util.memory.ZKQueueMemoryAllocationUtilities; import org.apache.drill.exec.work.foreman.ForemanSetupException; import org.apache.drill.shaded.guava.com.google.common.collect.Lists; diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/ZKQueueParallelizer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/ZKQueueParallelizer.java new file mode 100644 index 00000000000..28cd3256f92 --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/ZKQueueParallelizer.java @@ -0,0 +1,121 @@ +/* + * 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.planner.fragment; + +import org.apache.drill.common.util.function.CheckedConsumer; +import org.apache.drill.exec.ops.QueryContext; +import org.apache.drill.exec.physical.PhysicalOperatorSetupException; +import org.apache.drill.exec.physical.base.Exchange; +import org.apache.drill.exec.physical.base.PhysicalOperator; +import org.apache.drill.exec.planner.AbstractOpWrapperVisitor; +import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint; +import org.apache.drill.exec.util.memory.ZKQueueMemoryAllocationUtilities; +import org.apache.drill.exec.work.foreman.rm.QueryResourceManager; +import org.apache.drill.shaded.guava.com.google.common.collect.ArrayListMultimap; +import org.apache.drill.shaded.guava.com.google.common.collect.Multimap; +import java.util.Collection; +import java.util.Map; +import java.util.HashMap; +import java.util.Set; +import java.util.function.BiFunction; + +public class ZKQueueParallelizer extends SimpleParallelizer { + + private final boolean planHasMemory; + private final QueryContext queryContext; + private Map> endpointMap; + private final QueryResourceManager resourceManager; + + public ZKQueueParallelizer(boolean memoryAvailableInPlan, QueryResourceManager rm, QueryContext queryContext) { + super(queryContext); + this.planHasMemory = memoryAvailableInPlan; + this.queryContext = queryContext; + this.resourceManager = rm; + } + + @Override + public void adjustMemory(PlanningSet planningSet, Set roots, + Collection activeEndpoints) throws PhysicalOperatorSetupException { + if (planHasMemory) { + return; + } + + Collector collector = new Collector(); + + for (Wrapper wrapper : roots) { + traverse(wrapper, CheckedConsumer.throwingConsumerWrapper((Wrapper fragment) -> { + fragment.getNode().getRoot().accept(collector, fragment); + })); + } + + endpointMap = collector.getNodeMap(); + + ZKQueueMemoryAllocationUtilities.planMemory(queryContext, this.resourceManager, endpointMap); + } + + + public class Collector extends AbstractOpWrapperVisitor { + private final Multimap bufferedOperators; + + public Collector() { + this.bufferedOperators = ArrayListMultimap.create(); + } + + private void getMinorFragCountPerDrillbit(Wrapper currFragment, PhysicalOperator operator) { + for (DrillbitEndpoint endpoint : currFragment.getAssignedEndpoints()) { + bufferedOperators.put(endpoint, operator); + } + } + + @Override + public Void visitSendingExchange(Exchange exchange, Wrapper fragment) throws RuntimeException { + return visitOp(exchange, fragment); + } + + @Override + public Void visitReceivingExchange(Exchange exchange, Wrapper fragment) throws RuntimeException { + return null; + } + + @Override + public Void visitOp(PhysicalOperator op, Wrapper fragment) { + if (op.isBufferedOperator(queryContext)) { + getMinorFragCountPerDrillbit(fragment, op); + } + for (PhysicalOperator child : op) { + child.accept(this, fragment); + } + return null; + } + + public Map> getNodeMap() { + Map> endpointCollectionMap = bufferedOperators.asMap(); + Map> nodeMap = new HashMap<>(); + for (Map.Entry> entry : endpointCollectionMap.entrySet()) { + nodeMap.put(entry.getKey().getAddress(), entry.getValue()); + } + + return nodeMap; + } + } + + @Override + protected BiFunction getMemory() { + return (endpoint, operator) -> operator.getMaxAllocation(); + } +} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/contrib/SplittingParallelizer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/contrib/SplittingParallelizer.java index c1250e3ac6a..08aaca71840 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/contrib/SplittingParallelizer.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/contrib/SplittingParallelizer.java @@ -29,7 +29,7 @@ import org.apache.drill.exec.physical.base.FragmentRoot; import org.apache.drill.exec.physical.base.PhysicalOperator; import org.apache.drill.exec.planner.PhysicalPlanReader; -import org.apache.drill.exec.planner.fragment.DefaultQueryParallelizer; +import org.apache.drill.exec.planner.fragment.DefaultParallelizer; import org.apache.drill.exec.planner.fragment.Fragment; import org.apache.drill.exec.planner.fragment.PlanningSet; import org.apache.drill.exec.planner.fragment.Wrapper; @@ -57,7 +57,7 @@ * allows not to pollute parent class with non-authentic functionality * */ -public class SplittingParallelizer extends DefaultQueryParallelizer { +public class SplittingParallelizer extends DefaultParallelizer { static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SplittingParallelizer.class); diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/util/MemoryAllocationUtilities.java b/exec/java-exec/src/main/java/org/apache/drill/exec/util/memory/DefaultMemoryAllocationUtilities.java similarity index 97% rename from exec/java-exec/src/main/java/org/apache/drill/exec/util/MemoryAllocationUtilities.java rename to exec/java-exec/src/main/java/org/apache/drill/exec/util/memory/DefaultMemoryAllocationUtilities.java index 008c9dfc0dc..648276ef201 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/util/MemoryAllocationUtilities.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/util/memory/DefaultMemoryAllocationUtilities.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.drill.exec.util; +package org.apache.drill.exec.util.memory; import java.util.ArrayList; import java.util.List; @@ -31,9 +31,9 @@ import org.apache.drill.shaded.guava.com.google.common.annotations.VisibleForTesting; -public class MemoryAllocationUtilities { +public class DefaultMemoryAllocationUtilities { - private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MemoryAllocationUtilities.class); + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DefaultMemoryAllocationUtilities.class); public static void setupBufferedMemoryAllocations(PhysicalPlan plan, final QueryContext queryContext) { diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/util/memory/ZKQueueMemoryAllocationUtilities.java b/exec/java-exec/src/main/java/org/apache/drill/exec/util/memory/ZKQueueMemoryAllocationUtilities.java new file mode 100644 index 00000000000..6d232596bdf --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/util/memory/ZKQueueMemoryAllocationUtilities.java @@ -0,0 +1,256 @@ +/* + * 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.util.memory; + +import org.apache.drill.common.config.DrillConfig; +import org.apache.drill.exec.ExecConstants; +import org.apache.drill.exec.memory.RootAllocatorFactory; +import org.apache.drill.exec.ops.QueryContext; +import org.apache.drill.exec.physical.base.AbstractPhysicalVisitor; +import org.apache.drill.exec.physical.base.FragmentRoot; +import org.apache.drill.exec.physical.base.PhysicalOperator; +import org.apache.drill.exec.proto.helper.QueryIdHelper; +import org.apache.drill.exec.server.options.OptionSet; +import org.apache.drill.exec.work.QueryWorkUnit; +import org.apache.drill.exec.work.foreman.rm.QueryResourceManager; +import org.apache.drill.shaded.guava.com.google.common.annotations.VisibleForTesting; +import org.apache.drill.shaded.guava.com.google.common.collect.ArrayListMultimap; +import org.apache.drill.shaded.guava.com.google.common.collect.Multimap; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Map; + +public class ZKQueueMemoryAllocationUtilities { + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ZKQueueMemoryAllocationUtilities.class); + + public static void planMemory(QueryContext queryContext, QueryResourceManager rm, Map> nodeMap) { + + // Memory must be symmetric to avoid bottlenecks in which one node has + // sorts (say) with less memory than another, causing skew in data arrival + // rates for downstream operators. + + int width = countBufferingOperators(nodeMap); + + // Then, share memory evenly across the + // all sort operators on that node. This handles asymmetric distribution + // such as occurs if a sort appears in the root fragment (the one with + // screen), + // which is never parallelized. + + for (Map.Entry> entry : nodeMap.entrySet()) { + planNodeMemory(entry.getKey(), queryContext, entry.getValue(), width, rm); + } + } + + public static void getBufferedOps(Multimap map, + QueryWorkUnit.MinorFragmentDefn defn) { + List bufferedOps = getBufferedOps(defn.root()); + if (!bufferedOps.isEmpty()) { + map.putAll(defn.fragment().getAssignment().getAddress(), bufferedOps); + } + } + + public static class BufferedOpFinder extends AbstractPhysicalVisitor, RuntimeException> { + @Override + public Void visitOp(PhysicalOperator op, List value) + throws RuntimeException { + if (op.isBufferedOperator(null)) { + value.add(op); + } + visitChildren(op, value); + return null; + } + } + + /** + * Search an individual fragment tree to find any buffered operators it may + * contain. + * + * @param root + * @return + */ + + private static List getBufferedOps(FragmentRoot root) { + List bufferedOps = new ArrayList<>(); + BufferedOpFinder finder = new BufferedOpFinder(); + root.accept(finder, bufferedOps); + return bufferedOps; + } + + /** + * Build a list of external sorts grouped by node. We start with a list of + * minor fragments, each with an endpoint (node). Multiple minor fragments + * may appear on each node, and each minor fragment may have 0, 1 or more + * sorts. + * + * @return + */ + + private static Map> buildBufferedOpMap(QueryWorkUnit work) { + Multimap map = ArrayListMultimap.create(); + getBufferedOps(map, work.getRootFragmentDefn()); + for (QueryWorkUnit.MinorFragmentDefn defn : work.getMinorFragmentDefns()) { + getBufferedOps(map, defn); + } + return map.asMap(); + } + + private static int countBufferingOperators(Map> nodeMap) { + int width = 0; + for (Collection fragSorts : nodeMap.values()) { + width = Math.max(width, fragSorts.size()); + } + return width; + } + + /** + * Given the set of buffered operators (from any number of fragments) on a + * single node, shared the per-query memory equally across all the + * operators. + * + * @param nodeAddr + * @param bufferedOps + * @param width + */ + + private static void planNodeMemory(String nodeAddr, QueryContext queryContext, + Collection bufferedOps, int width, + QueryResourceManager rm) { + + // If no buffering operators, nothing to plan. + + if (bufferedOps.isEmpty()) { + return; + } + + // Divide node memory evenly among the set of operators, in any minor + // fragment, on the node. This is not very sophisticated: it does not + // deal with, say, three stacked sorts in which, if sort A runs, then + // B may be using memory, but C cannot be active. That kind of analysis + // is left as a later exercise. + + // Set a floor on the amount of memory per operator based on the + // configured minimum. This is likely unhelpful because we are trying + // to work around constrained memory by assuming more than we actually + // have. This may lead to an OOM at run time. + + long preferredOpMemory = rm.queryMemoryPerNode()/ width; + long perOpMemory = Math.max(preferredOpMemory, rm.minimumOperatorMemory()); + if (preferredOpMemory < perOpMemory) { + logger.warn("Preferred per-operator memory: {}, actual amount: {}", + preferredOpMemory, perOpMemory); + } + logger.debug( + "Query: {}, Node: {}, allocating {} bytes each for {} buffered operator(s).", + QueryIdHelper.getQueryId(queryContext.getQueryId()), nodeAddr, + perOpMemory, width); + + for (PhysicalOperator op : bufferedOps) { + + // Limit the memory to the maximum in the plan. Doing so is + // likely unnecessary, and perhaps harmful, because the pre-planned + // allocation is the default maximum hard-coded to 10 GB. This means + // that even if 20 GB is available to the sort, it won't use more + // than 10GB. This is probably more of a bug than a feature. + + long alloc = Math.min(perOpMemory, op.getMaxAllocation()); + + // Place a floor on the memory that is the initial allocation, + // since we don't want the operator to run out of memory when it + // first starts. + + alloc = Math.max(alloc, op.getInitialAllocation()); + + if (alloc > preferredOpMemory && alloc != perOpMemory) { + logger.warn("Allocated memory of {} for {} exceeds available memory of {} " + + "due to operator minimum", + alloc, op.getClass().getSimpleName(), preferredOpMemory); + } + else if (alloc < preferredOpMemory) { + logger.warn("Allocated memory of {} for {} is less than available memory " + + "of {} due to operator limit", + alloc, op.getClass().getSimpleName(), preferredOpMemory); + } + op.setMaxAllocation(alloc); + } + } + + /** + * Compute per-operator memory based on the computed per-node memory, the + * number of operators, and the computed number of fragments (which house + * the operators.) Enforces a floor on the amount of memory per operator. + * + * @param optionManager system option manager + * @param maxAllocPerNode computed query memory per node + * @param opCount number of buffering operators in this query + * @return the per-operator memory + */ + + public static long computeOperatorMemory(OptionSet optionManager, long maxAllocPerNode, int opCount) { + final long maxWidth = optionManager.getOption(ExecConstants.MAX_WIDTH_PER_NODE); + final double cpuLoadAverage = optionManager.getOption(ExecConstants.CPU_LOAD_AVERAGE); + final long maxWidthPerNode = ExecConstants.MAX_WIDTH_PER_NODE.computeMaxWidth(cpuLoadAverage, maxWidth); + final long maxOperatorAlloc = maxAllocPerNode / (opCount * maxWidthPerNode); + logger.debug("Max buffered operator alloc: {}", maxOperatorAlloc); + + // User configurable option to allow forcing minimum memory. + // Ensure that the buffered ops receive the minimum memory needed to make progress. + // Without this, the math might work out to allocate too little memory. + + return Math.max(maxOperatorAlloc, + optionManager.getOption(ExecConstants.MIN_MEMORY_PER_BUFFERED_OP)); + } + + /** + * Per-node memory calculations based on a number of constraints. + *

+ * Factored out into a separate method to allow unit testing. + * @param config Drill config + * @param optionManager system options + * @param directMemory amount of direct memory + * @return memory per query per node + */ + + @VisibleForTesting + public static long computeQueryMemory(DrillConfig config, OptionSet optionManager, long directMemory) { + + // Memory computed as a percent of total memory. + + long perQueryMemory = Math.round(directMemory * + optionManager.getOption(ExecConstants.PERCENT_MEMORY_PER_QUERY)); + + // But, must allow at least the amount given explicitly for + // backward compatibility. + + perQueryMemory = Math.max(perQueryMemory, + optionManager.getOption(ExecConstants.MAX_QUERY_MEMORY_PER_NODE)); + + // Compute again as either the total direct memory, or the + // configured maximum top-level allocation (10 GB). + + long maxAllocPerNode = Math.min(directMemory, + config.getLong(RootAllocatorFactory.TOP_LEVEL_MAX_ALLOC)); + + // Final amount per node per query is the minimum of these two. + + maxAllocPerNode = Math.min(maxAllocPerNode, perQueryMemory); + return maxAllocPerNode; + } +} 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 ce03303a896..df690444b3a 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 @@ -412,7 +412,6 @@ private void runPhysicalPlan(final PhysicalPlan plan) throws ExecutionSetupExcep private void runPhysicalPlan(final PhysicalPlan plan, Pointer textPlan) throws ExecutionSetupException { validatePlan(plan); - queryRM.visitAbstractPlan(plan); final QueryWorkUnit work = getQueryWorkUnit(plan, queryRM); if (enableRuntimeFilter) { runtimeFilterRouter = new RuntimeFilterRouter(work, drillbitContext); @@ -421,7 +420,6 @@ private void runPhysicalPlan(final PhysicalPlan plan, Pointer textPlan) if (textPlan != null) { queryManager.setPlanText(textPlan.value); } - queryRM.visitPhysicalPlan(work); queryRM.setCost(plan.totalCost()); queryManager.setTotalCost(plan.totalCost()); work.applyPlan(drillbitContext.getPlanReader()); 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 7c3cc04d5e9..5c0f71d8d8c 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,11 +19,8 @@ import org.apache.drill.common.config.DrillConfig; import org.apache.drill.exec.ops.QueryContext; -import org.apache.drill.exec.physical.PhysicalPlan; import org.apache.drill.exec.planner.fragment.QueryParallelizer; -import org.apache.drill.exec.planner.fragment.DefaultQueryParallelizer; -import org.apache.drill.exec.util.MemoryAllocationUtilities; -import org.apache.drill.exec.work.QueryWorkUnit; +import org.apache.drill.exec.planner.fragment.DefaultParallelizer; import org.apache.drill.exec.work.foreman.Foreman; /** @@ -35,39 +32,14 @@ public class DefaultResourceManager implements ResourceManager { - public static class DefaultResourceAllocator implements QueryResourceAllocator { + public static class DefaultQueryResourceManager implements QueryResourceManager { - private QueryContext queryContext; - - protected DefaultResourceAllocator(QueryContext queryContext) { - this.queryContext = queryContext; - } - - @Override - public void visitAbstractPlan(PhysicalPlan plan) { - if (plan == null || plan.getProperties().hasResourcePlan) { - return; - } - MemoryAllocationUtilities.setupBufferedMemoryAllocations(plan, queryContext); - } - - @Override - public void visitPhysicalPlan(QueryWorkUnit work) { - } - - public QueryContext getQueryContext() { - return queryContext; - } - } - - public static class DefaultQueryResourceManager extends DefaultResourceAllocator implements QueryResourceManager { - - @SuppressWarnings("unused") private final DefaultResourceManager rm; + private final QueryContext queryContext; public DefaultQueryResourceManager(final DefaultResourceManager rm, final Foreman foreman) { - super(foreman.getQueryContext()); this.rm = rm; + this.queryContext = foreman.getQueryContext(); } @Override @@ -77,7 +49,7 @@ public void setCost(double cost) { @Override public QueryParallelizer getParallelizer(boolean memoryPlanning){ - return new DefaultQueryParallelizer(memoryPlanning, this.getQueryContext()); + return new DefaultParallelizer(memoryPlanning, queryContext); } @Override @@ -95,6 +67,16 @@ public void exit() { @Override public String queueName() { return null; } + + @Override + public long queryMemoryPerNode() { + return rm.memoryPerNode; + } + + @Override + public long minimumOperatorMemory() { + return 0; + } } public final long memoryPerNode; @@ -115,11 +97,6 @@ public DefaultResourceManager() { @Override public int cpusPerNode() { return cpusPerNode; } - @Override - public QueryResourceAllocator newResourceAllocator(QueryContext queryContext) { - return new DefaultResourceAllocator(queryContext); - } - @Override public QueryResourceManager newQueryRM(final Foreman foreman) { return new DefaultQueryResourceManager(this, foreman); 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 af9ba00f4ce..6932bcdb20e 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 @@ -21,6 +21,8 @@ import org.apache.drill.common.exceptions.DrillRuntimeException; import org.apache.drill.exec.ExecConstants; import org.apache.drill.exec.ops.QueryContext; +import org.apache.drill.exec.planner.fragment.QueryParallelizer; +import org.apache.drill.exec.planner.fragment.DistributedQueueParallelizer; import org.apache.drill.exec.resourcemgr.config.ResourcePoolTree; import org.apache.drill.exec.resourcemgr.config.ResourcePoolTreeImpl; import org.apache.drill.exec.resourcemgr.config.exception.RMConfigException; @@ -35,17 +37,18 @@ public class DistributedResourceManager implements ResourceManager { private final DrillbitContext context; private final DrillConfig rmConfig; - - private final ResourceManager delegatedRM; + public final long memoryPerNode; + public final int cpusPerNode; public DistributedResourceManager(DrillbitContext context) throws DrillRuntimeException { + memoryPerNode = DrillConfig.getMaxDirectMemory(); + cpusPerNode = Runtime.getRuntime().availableProcessors(); try { this.context = context; this.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)); - this.delegatedRM = new DefaultResourceManager(); } 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); @@ -53,30 +56,84 @@ public DistributedResourceManager(DrillbitContext context) throws DrillRuntimeEx } @Override public long memoryPerNode() { - return delegatedRM.memoryPerNode(); + return memoryPerNode; } @Override public int cpusPerNode() { - return delegatedRM.cpusPerNode(); - } - - @Override - public QueryResourceAllocator newResourceAllocator(QueryContext queryContext) { - return delegatedRM.newResourceAllocator(queryContext); + return cpusPerNode; } @Override public QueryResourceManager newQueryRM(Foreman foreman) { - return delegatedRM.newQueryRM(foreman); + return new QueuedQueryResourceManager(this, foreman); } public ResourcePoolTree getRmPoolTree() { return rmPoolTree; } + + /** + * 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 + * with a Foreman: a single thread at plan time, and a single event (in some + * thread) at query completion time. Because of these semantics, no + * synchronization is needed within this class. + */ + + public static class QueuedQueryResourceManager implements QueryResourceManager { + + private final Foreman foreman; + private final QueryContext queryContext; + private double queryCost; + private final DistributedResourceManager rm; + + public QueuedQueryResourceManager(final DistributedResourceManager rm, + final Foreman foreman) { + this.foreman = foreman; + this.queryContext = foreman.getQueryContext(); + this.rm = rm; + } + + @Override + public void setCost(double cost) { + this.queryCost = cost; + } + + @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.queryContext); + } + + @Override + public void admit() throws QueryQueue.QueueTimeoutException, QueryQueue.QueryQueueException { + } + + public long queryMemoryPerNode() { + return 0; + } + + @Override + public long minimumOperatorMemory() { + return 0; + } + + @Override + public void exit() { + } + + @Override + public boolean hasQueue() { return true; } + + @Override + public String queueName() { + return ""; + } + } + @Override public void close() { - delegatedRM.close(); } } 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 7491e602275..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 @@ -19,7 +19,6 @@ import org.apache.drill.common.exceptions.UserException; import org.apache.drill.exec.ExecConstants; -import org.apache.drill.exec.ops.QueryContext; import org.apache.drill.exec.server.DrillbitContext; import org.apache.drill.exec.server.options.SystemOptionManager; import org.apache.drill.exec.work.foreman.Foreman; @@ -61,12 +60,6 @@ public int cpusPerNode() { return activeRm.cpusPerNode(); } - @Override - public synchronized QueryResourceAllocator newResourceAllocator(QueryContext queryContext) { - refreshRM(); - return activeRm.newResourceAllocator(queryContext); - } - @Override public synchronized QueryResourceManager newQueryRM(Foreman foreman) { refreshRM(); diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/QueryResourceAllocator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/QueryResourceAllocator.java deleted file mode 100644 index 35dbe59879c..00000000000 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/QueryResourceAllocator.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.drill.exec.work.foreman.rm; - -import org.apache.drill.exec.physical.PhysicalPlan; -import org.apache.drill.exec.work.QueryWorkUnit; - -/** - * Manages resources for an individual query in conjunction with the - * global {@link ResourceManager}. Handles memory and CPU allocation. - * Instances of this class handle query planning and are used when the - * client wants to plan the query, but not execute it. An implementation - * of {@link QueryResourceManager} is used to both plan the query and - * queue it for execution. - *

- * This interface allows a variety of resource management strategies to - * exist for different purposes. - *

- * The methods here assume external synchronization: a single query calls - * the methods at known times; there are no concurrent calls. - */ - -public interface QueryResourceAllocator { - - /** - * Make any needed adjustments to the query plan before parallelization. - * - * @param plan - */ - void visitAbstractPlan(PhysicalPlan plan); - - /** - * Provide the manager with the physical plan and node assignments - * for the query to be run. This class will plan memory for the query. - * - * @param plan - * @param work - */ - - void visitPhysicalPlan(QueryWorkUnit work); -} 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 4b9112194ce..a741c97ce49 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 @@ -25,7 +25,7 @@ * Extends a {@link QueryResourceAllocator} to provide queueing support. */ -public interface QueryResourceManager extends QueryResourceAllocator { +public interface QueryResourceManager { /** * Hint that this resource manager queues. Allows the Foreman @@ -65,6 +65,7 @@ public interface QueryResourceManager extends QueryResourceAllocator { void 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. @@ -74,6 +75,11 @@ public interface QueryResourceManager extends QueryResourceAllocator { String queueName(); + + long queryMemoryPerNode(); + + long minimumOperatorMemory(); + /** * Mark the query as completing, giving up its slot in the * cluster. Releases any lease that may be held for a system with queues. 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 71dabafa7ba..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 @@ -17,7 +17,6 @@ */ package org.apache.drill.exec.work.foreman.rm; -import org.apache.drill.exec.ops.QueryContext; import org.apache.drill.exec.work.foreman.Foreman; /** @@ -42,16 +41,6 @@ public interface ResourceManager { int cpusPerNode(); - /** - * Create a resource manager to prepare or describe a query. In this form, no - * queuing is done, but the plan is created as if queuing had been done. Used - * when executing EXPLAIN PLAN. - * - * @return a resource manager for the query - */ - - QueryResourceAllocator newResourceAllocator(QueryContext queryContext); - /** * Create a resource manager to execute a query. * 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 5d1a1709dd0..3572dd7bdb1 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 @@ -17,31 +17,15 @@ */ package org.apache.drill.exec.work.foreman.rm; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; - import org.apache.drill.exec.ops.QueryContext; -import org.apache.drill.exec.physical.PhysicalPlan; -import org.apache.drill.exec.physical.base.AbstractPhysicalVisitor; -import org.apache.drill.exec.physical.base.FragmentRoot; -import org.apache.drill.exec.physical.base.PhysicalOperator; +import org.apache.drill.exec.planner.fragment.ZKQueueParallelizer; import org.apache.drill.exec.planner.fragment.QueryParallelizer; -import org.apache.drill.exec.planner.fragment.QueueQueryParallelizer; -import org.apache.drill.exec.proto.helper.QueryIdHelper; import org.apache.drill.exec.server.DrillbitContext; -import org.apache.drill.exec.work.QueryWorkUnit; -import org.apache.drill.exec.work.QueryWorkUnit.MinorFragmentDefn; 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.QueueLease; import org.apache.drill.exec.work.foreman.rm.QueryQueue.QueueTimeoutException; -import org.apache.drill.shaded.guava.com.google.common.collect.ArrayListMultimap; -import org.apache.drill.shaded.guava.com.google.common.collect.Multimap; - /** * Global resource manager that provides basic admission control (AC) via a * configured queue: either the Zookeeper-based distributed queue or the @@ -61,211 +45,7 @@ public class ThrottledResourceManager extends AbstractResourceManager { - private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory - .getLogger(ThrottledResourceManager.class); - - public static class QueuedResourceAllocator - implements QueryResourceAllocator { - - protected final ThrottledResourceManager rm; - protected QueryContext queryContext; - protected PhysicalPlan plan; - protected QueryWorkUnit work; - protected double queryCost; - - protected QueuedResourceAllocator(final ThrottledResourceManager rm, - QueryContext queryContext) { - this.rm = rm; - this.queryContext = queryContext; - } - - @Override - public void visitAbstractPlan(PhysicalPlan plan) { - this.plan = plan; - queryCost = plan.totalCost(); - } - - @Override - public void visitPhysicalPlan(final QueryWorkUnit work) { - this.work = work; - planMemory(); - } - - private void planMemory() { - if (plan.getProperties().hasResourcePlan) { - logger.debug("Memory already planned."); - return; - } - - // Group fragments by node. - - Map> nodeMap = buildBufferedOpMap(); - - // Memory must be symmetric to avoid bottlenecks in which one node has - // sorts (say) with less memory than another, causing skew in data arrival - // rates for downstream operators. - - int width = countBufferingOperators(nodeMap); - - // Then, share memory evenly across the - // all sort operators on that node. This handles asymmetric distribution - // such as occurs if a sort appears in the root fragment (the one with - // screen), - // which is never parallelized. - - for (Entry> entry : nodeMap.entrySet()) { - planNodeMemory(entry.getKey(), entry.getValue(), width); - } - } - - public QueryContext getQueryContext() { - return queryContext; - } - - private int countBufferingOperators( - Map> nodeMap) { - int width = 0; - for (Collection fragSorts : nodeMap.values()) { - width = Math.max(width, fragSorts.size()); - } - return width; - } - - /** - * Given the set of buffered operators (from any number of fragments) on a - * single node, shared the per-query memory equally across all the - * operators. - * - * @param nodeAddr - * @param bufferedOps - * @param width - */ - - private void planNodeMemory(String nodeAddr, - Collection bufferedOps, int width) { - - // If no buffering operators, nothing to plan. - - if (bufferedOps.isEmpty()) { - return; - } - - // Divide node memory evenly among the set of operators, in any minor - // fragment, on the node. This is not very sophisticated: it does not - // deal with, say, three stacked sorts in which, if sort A runs, then - // B may be using memory, but C cannot be active. That kind of analysis - // is left as a later exercise. - - long nodeMemory = queryMemoryPerNode(); - - // Set a floor on the amount of memory per operator based on the - // configured minimum. This is likely unhelpful because we are trying - // to work around constrained memory by assuming more than we actually - // have. This may lead to an OOM at run time. - - long preferredOpMemory = nodeMemory / width; - long perOpMemory = Math.max(preferredOpMemory, rm.minimumOperatorMemory()); - if (preferredOpMemory < perOpMemory) { - logger.warn("Preferred per-operator memory: {}, actual amount: {}", - preferredOpMemory, perOpMemory); - } - logger.debug( - "Query: {}, Node: {}, allocating {} bytes each for {} buffered operator(s).", - QueryIdHelper.getQueryId(queryContext.getQueryId()), nodeAddr, - perOpMemory, width); - - for (PhysicalOperator op : bufferedOps) { - - // Limit the memory to the maximum in the plan. Doing so is - // likely unnecessary, and perhaps harmful, because the pre-planned - // allocation is the default maximum hard-coded to 10 GB. This means - // that even if 20 GB is available to the sort, it won't use more - // than 10GB. This is probably more of a bug than a feature. - - long alloc = Math.min(perOpMemory, op.getMaxAllocation()); - - // Place a floor on the memory that is the initial allocation, - // since we don't want the operator to run out of memory when it - // first starts. - - alloc = Math.max(alloc, op.getInitialAllocation()); - - if (alloc > preferredOpMemory && alloc != perOpMemory) { - logger.warn("Allocated memory of {} for {} exceeds available memory of {} " + - "due to operator minimum", - alloc, op.getClass().getSimpleName(), preferredOpMemory); - } - else if (alloc < preferredOpMemory) { - logger.warn("Allocated memory of {} for {} is less than available memory " + - "of {} due to operator limit", - alloc, op.getClass().getSimpleName(), preferredOpMemory); - } - op.setMaxAllocation(alloc); - } - } - - protected long queryMemoryPerNode() { - return rm.defaultQueryMemoryPerNode(plan.totalCost()); - } - - /** - * Build a list of external sorts grouped by node. We start with a list of - * minor fragments, each with an endpoint (node). Multiple minor fragments - * may appear on each node, and each minor fragment may have 0, 1 or more - * sorts. - * - * @return - */ - - private Map> buildBufferedOpMap() { - Multimap map = ArrayListMultimap.create(); - getBufferedOps(map, work.getRootFragmentDefn()); - for (MinorFragmentDefn defn : work.getMinorFragmentDefns()) { - getBufferedOps(map, defn); - } - return map.asMap(); - } - - /** - * Searches a fragment operator tree to find buffered within that fragment. - */ - - protected static class BufferedOpFinder extends - AbstractPhysicalVisitor, RuntimeException> { - @Override - public Void visitOp(PhysicalOperator op, List value) - throws RuntimeException { - if (op.isBufferedOperator(null)) { - value.add(op); - } - visitChildren(op, value); - return null; - } - } - - private void getBufferedOps(Multimap map, - MinorFragmentDefn defn) { - List bufferedOps = getBufferedOps(defn.root()); - if (!bufferedOps.isEmpty()) { - map.putAll(defn.fragment().getAssignment().getAddress(), bufferedOps); - } - } - - /** - * Search an individual fragment tree to find any buffered operators it may - * contain. - * - * @param root - * @return - */ - - private List getBufferedOps(FragmentRoot root) { - List bufferedOps = new ArrayList<>(); - BufferedOpFinder finder = new BufferedOpFinder(); - root.accept(finder, bufferedOps); - return bufferedOps; - } - } + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ThrottledResourceManager.class); /** * Per-query resource manager. Handles resources and optional queue lease for @@ -275,16 +55,19 @@ private List getBufferedOps(FragmentRoot root) { * synchronization is needed within this class. */ - public static class QueuedQueryResourceManager extends QueuedResourceAllocator - implements QueryResourceManager { + public static class QueuedQueryResourceManager implements QueryResourceManager { private final Foreman foreman; private QueueLease lease; + private final QueryContext queryContext; + private double queryCost; + private final ThrottledResourceManager rm; public QueuedQueryResourceManager(final ThrottledResourceManager rm, final Foreman foreman) { - super(rm, foreman.getQueryContext()); this.foreman = foreman; + this.queryContext = foreman.getQueryContext(); + this.rm = rm; } @Override @@ -295,7 +78,7 @@ public void setCost(double cost) { @Override public QueryParallelizer getParallelizer(boolean planHasMemory) { // currently memory planning is disabled. Enable it once the RM functionality is fully implemented. - return new QueueQueryParallelizer(true || planHasMemory, this.getQueryContext()); + return new ZKQueueParallelizer(planHasMemory, this, this.queryContext); } @Override @@ -303,13 +86,10 @@ public void admit() throws QueueTimeoutException, QueryQueueException { lease = rm.queue().enqueue(foreman.getQueryId(), queryCost); } - @Override - protected long queryMemoryPerNode() { - - // No lease: use static estimate. + public long queryMemoryPerNode() { if (lease == null) { - return super.queryMemoryPerNode(); + return rm.defaultQueryMemoryPerNode(queryCost); } // Use actual memory assigned to this query. @@ -317,6 +97,11 @@ protected long queryMemoryPerNode() { return lease.queryMemoryPerNode(); } + @Override + public long minimumOperatorMemory() { + return rm.minimumOperatorMemory(); + } + @Override public void exit() { if (lease != null) { @@ -353,12 +138,6 @@ public long defaultQueryMemoryPerNode(double cost) { public QueryQueue queue() { return queue; } - @Override - public QueryResourceAllocator newResourceAllocator( - QueryContext queryContext) { - return new QueuedResourceAllocator(this, queryContext); - } - @Override public QueryResourceManager newQueryRM(Foreman foreman) { return new QueuedQueryResourceManager(this, foreman); diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/user/PlanSplitter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/user/PlanSplitter.java index e907c54bd58..66f4dfd27bb 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/user/PlanSplitter.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/user/PlanSplitter.java @@ -38,7 +38,6 @@ import org.apache.drill.exec.server.DrillbitContext; import org.apache.drill.exec.util.Pointer; import org.apache.drill.exec.work.QueryWorkUnit; -import org.apache.drill.exec.work.foreman.rm.QueryResourceAllocator; import org.apache.drill.shaded.guava.com.google.common.collect.Lists; @@ -106,9 +105,6 @@ private List getFragments(final DrillbitContext dContext, final Ge throw new IllegalStateException("Planning fragments supports only SQL or PHYSICAL QueryType"); } - QueryResourceAllocator planner = dContext.getResourceManager().newResourceAllocator(queryContext); - planner.visitAbstractPlan(plan); - final PhysicalOperator rootOperator = plan.getSortedOperators(false).iterator().next(); final Fragment rootFragment = rootOperator.accept(MakeFragmentsVisitor.INSTANCE, null); @@ -123,7 +119,7 @@ private List getFragments(final DrillbitContext dContext, final Ge queryContext.getSession(), queryContext.getQueryContextInfo()); for (QueryWorkUnit queryWorkUnit : queryWorkUnits) { - planner.visitPhysicalPlan(queryWorkUnit); + queryWorkUnit.applyPlan(dContext.getPlanReader()); fragments.add(queryWorkUnit.getRootFragment()); @@ -136,7 +132,7 @@ private List getFragments(final DrillbitContext dContext, final Ge final QueryWorkUnit queryWorkUnit = parallelizer.generateWorkUnit(queryContext.getOptions().getOptionList(), queryContext.getCurrentEndpoint(), queryId, queryContext.getActiveEndpoints(), rootFragment, queryContext.getSession(), queryContext.getQueryContextInfo()); - planner.visitPhysicalPlan(queryWorkUnit); +// planner.visitPhysicalPlan(queryWorkUnit); queryWorkUnit.applyPlan(dContext.getPlanReader()); fragments.add(queryWorkUnit.getRootFragment()); fragments.addAll(queryWorkUnit.getFragments()); diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestLocalExchange.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestLocalExchange.java index 37cae3e92d1..6e508ac031a 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestLocalExchange.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestLocalExchange.java @@ -17,7 +17,7 @@ */ package org.apache.drill.exec.physical.impl; -import org.apache.drill.exec.planner.fragment.DefaultQueryParallelizer; +import org.apache.drill.exec.planner.fragment.DefaultParallelizer; import org.apache.drill.shaded.guava.com.google.common.collect.ArrayListMultimap; import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableSet; import org.apache.drill.shaded.guava.com.google.common.collect.Lists; @@ -82,7 +82,7 @@ public class TestLocalExchange extends PlanTestBase { .withCredentials(UserBitShared.UserCredentials.newBuilder().setUserName("foo").build()) .build(); - private static final SimpleParallelizer PARALLELIZER = new DefaultQueryParallelizer( + private static final SimpleParallelizer PARALLELIZER = new DefaultParallelizer( true, 1 /*parallelizationThreshold (slice_count)*/, 6 /*maxWidthPerNode*/, diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/partitionsender/TestPartitionSender.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/partitionsender/TestPartitionSender.java index 7ef94f7d760..c246eba0069 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/partitionsender/TestPartitionSender.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/partitionsender/TestPartitionSender.java @@ -47,7 +47,7 @@ import org.apache.drill.exec.physical.impl.partitionsender.PartitionerDecorator.GeneralExecuteIface; import org.apache.drill.exec.planner.PhysicalPlanReader; import org.apache.drill.exec.planner.cost.PrelCostEstimates; -import org.apache.drill.exec.planner.fragment.DefaultQueryParallelizer; +import org.apache.drill.exec.planner.fragment.DefaultParallelizer; import org.apache.drill.exec.planner.fragment.Fragment; import org.apache.drill.exec.planner.fragment.PlanningSet; import org.apache.drill.exec.planner.fragment.SimpleParallelizer; @@ -88,7 +88,7 @@ @Category(OperatorTest.class) public class TestPartitionSender extends PlanTestBase { - private static final SimpleParallelizer PARALLELIZER = new DefaultQueryParallelizer( + private static final SimpleParallelizer PARALLELIZER = new DefaultParallelizer( true, 1 /*parallelizationThreshold (slice_count)*/, 6 /*maxWidthPerNode*/, 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 891b4a6a226..198a5290118 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 @@ -25,7 +25,7 @@ import org.apache.drill.exec.planner.cost.NodeResource; import org.apache.drill.exec.planner.fragment.Fragment; import org.apache.drill.exec.planner.fragment.PlanningSet; -import org.apache.drill.exec.planner.fragment.QueueQueryParallelizer; +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.pop.PopUnitTestBase; @@ -190,7 +190,7 @@ public void TestSingleMajorFragmentWithProjectAndScan() throws Exception { x -> NodeResource.create())); String sql = "SELECT * from cp.`tpch/nation.parquet`"; - SimpleParallelizer parallelizer = new QueueQueryParallelizer(false, queryContext); + SimpleParallelizer parallelizer = new DistributedQueueParallelizer(false, queryContext); PlanningSet planningSet = preparePlanningSet(activeEndpoints, DEFAULT_SLICE_TARGET, resources, sql, parallelizer); parallelizer.adjustMemory(planningSet, createSet(planningSet.getRootWrapper()), activeEndpoints); assertTrue("memory requirement is different", Iterables.all(resources.entrySet(), (e) -> e.getValue().getMemory() == 30)); @@ -205,7 +205,7 @@ public void TestSingleMajorFragmentWithGroupByProjectAndScan() throws Exception x -> NodeResource.create())); String sql = "SELECT dept_id, count(*) from cp.`tpch/lineitem.parquet` group by dept_id"; - SimpleParallelizer parallelizer = new QueueQueryParallelizer(false, queryContext); + SimpleParallelizer parallelizer = new DistributedQueueParallelizer(false, queryContext); PlanningSet planningSet = preparePlanningSet(activeEndpoints, DEFAULT_SLICE_TARGET, resources, sql, parallelizer); parallelizer.adjustMemory(planningSet, createSet(planningSet.getRootWrapper()), activeEndpoints); assertTrue("memory requirement is different", Iterables.all(resources.entrySet(), (e) -> e.getValue().getMemory() == 529570)); @@ -220,7 +220,7 @@ public void TestTwoMajorFragmentWithSortyProjectAndScan() throws Exception { x -> NodeResource.create())); String sql = "SELECT * from cp.`tpch/lineitem.parquet` order by dept_id"; - SimpleParallelizer parallelizer = new QueueQueryParallelizer(false, queryContext); + SimpleParallelizer parallelizer = new DistributedQueueParallelizer(false, queryContext); PlanningSet planningSet = preparePlanningSet(activeEndpoints, 2, resources, sql, parallelizer); parallelizer.adjustMemory(planningSet, createSet(planningSet.getRootWrapper()), activeEndpoints); assertTrue("memory requirement is different", Iterables.all(resources.entrySet(), (e) -> e.getValue().getMemory() == 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 c01143b26bb..f3b15884c45 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 @@ -22,7 +22,7 @@ import org.apache.drill.categories.PlannerTest; import org.apache.drill.exec.planner.PhysicalPlanReader; import org.apache.drill.exec.planner.PhysicalPlanReaderTestFactory; -import org.apache.drill.exec.planner.fragment.DefaultQueryParallelizer; +import org.apache.drill.exec.planner.fragment.DefaultParallelizer; import org.apache.drill.exec.planner.fragment.Fragment; import org.apache.drill.exec.planner.fragment.SimpleParallelizer; import org.apache.drill.exec.proto.BitControl.QueryContextInformation; @@ -53,7 +53,7 @@ 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 DefaultQueryParallelizer(true, 1000*1000, 5, 10, 1.2); + SimpleParallelizer par = new DefaultParallelizer(true, 1000*1000, 5, 10, 1.2); List endpoints = Lists.newArrayList(); DrillbitEndpoint localBit = null; for(int i =0; i < bitCount; i++) { diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/util/TestQueryMemoryAlloc.java b/exec/java-exec/src/test/java/org/apache/drill/exec/util/TestQueryMemoryAlloc.java index 14a9447f0fd..c8424608c13 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/util/TestQueryMemoryAlloc.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/util/TestQueryMemoryAlloc.java @@ -21,6 +21,7 @@ import org.apache.drill.exec.ExecConstants; import org.apache.drill.exec.server.options.OptionManager; +import org.apache.drill.exec.util.memory.ZKQueueMemoryAllocationUtilities; import org.apache.drill.test.BaseDirTestWatcher; import org.apache.drill.test.DrillTest; import org.apache.drill.test.OperatorFixture; @@ -48,17 +49,17 @@ public void testDefaultOptions() throws Exception { optionManager.setLocalOption(ExecConstants.MAX_QUERY_MEMORY_PER_NODE_KEY, 2 * ONE_GB); // Out-of-box memory, use query memory per node as floor. - long mem = MemoryAllocationUtilities.computeQueryMemory(fixture.config(), optionManager, 8 * ONE_GB); + long mem = ZKQueueMemoryAllocationUtilities.computeQueryMemory(fixture.config(), optionManager, 8 * ONE_GB); assertEquals(2 * ONE_GB, mem); // Up to 40 GB, query memory dominates. - mem = MemoryAllocationUtilities.computeQueryMemory(fixture.config(), optionManager, 40 * ONE_GB); + mem = ZKQueueMemoryAllocationUtilities.computeQueryMemory(fixture.config(), optionManager, 40 * ONE_GB); assertEquals(2 * ONE_GB, mem); // After 40 GB, the percent dominates - mem = MemoryAllocationUtilities.computeQueryMemory(fixture.config(), optionManager, 100 * ONE_GB); + mem = ZKQueueMemoryAllocationUtilities.computeQueryMemory(fixture.config(), optionManager, 100 * ONE_GB); assertEquals(5 * ONE_GB, mem); } } @@ -76,17 +77,17 @@ public void testCustomFloor() throws Exception { optionManager.setLocalOption(ExecConstants.MAX_QUERY_MEMORY_PER_NODE_KEY, 3 * ONE_GB); // Out-of-box memory, use query memory per node as floor. - long mem = MemoryAllocationUtilities.computeQueryMemory(fixture.config(), optionManager, 8 * ONE_GB); + long mem = ZKQueueMemoryAllocationUtilities.computeQueryMemory(fixture.config(), optionManager, 8 * ONE_GB); assertEquals(3 * ONE_GB, mem); // Up to 60 GB, query memory dominates. - mem = MemoryAllocationUtilities.computeQueryMemory(fixture.config(), optionManager, 60 * ONE_GB); + mem = ZKQueueMemoryAllocationUtilities.computeQueryMemory(fixture.config(), optionManager, 60 * ONE_GB); assertEquals(3 * ONE_GB, mem); // After 60 GB, the percent dominates - mem = MemoryAllocationUtilities.computeQueryMemory(fixture.config(), optionManager, 100 * ONE_GB); + mem = ZKQueueMemoryAllocationUtilities.computeQueryMemory(fixture.config(), optionManager, 100 * ONE_GB); assertEquals(5 * ONE_GB, mem); } } @@ -105,17 +106,17 @@ public void testCustomPercent() throws Exception { // Out-of-box memory, use query memory per node as floor. - long mem = MemoryAllocationUtilities.computeQueryMemory(fixture.config(), optionManager, 8 * ONE_GB); + long mem = ZKQueueMemoryAllocationUtilities.computeQueryMemory(fixture.config(), optionManager, 8 * ONE_GB); assertEquals(2 * ONE_GB, mem); // Up to 20 GB, query memory dominates. - mem = MemoryAllocationUtilities.computeQueryMemory(fixture.config(), optionManager, 20 * ONE_GB); + mem = ZKQueueMemoryAllocationUtilities.computeQueryMemory(fixture.config(), optionManager, 20 * ONE_GB); assertEquals(2 * ONE_GB, mem); // After 20 GB, the percent dominates - mem = MemoryAllocationUtilities.computeQueryMemory(fixture.config(), optionManager, 30 * ONE_GB); + mem = ZKQueueMemoryAllocationUtilities.computeQueryMemory(fixture.config(), optionManager, 30 * ONE_GB); assertEquals(3 * ONE_GB, mem); } } @@ -145,12 +146,12 @@ public void testOpMemory() throws Exception { // Enough memory to go above configured minimum. - long opMinMem = MemoryAllocationUtilities.computeOperatorMemory(optionManager, 4 * ONE_GB, 2); + long opMinMem = ZKQueueMemoryAllocationUtilities.computeOperatorMemory(optionManager, 4 * ONE_GB, 2); assertEquals(4 * ONE_GB / 10 / 2, opMinMem); // Too little memory per operator. Use configured minimum. - opMinMem = MemoryAllocationUtilities.computeOperatorMemory(optionManager, ONE_GB, 100); + opMinMem = ZKQueueMemoryAllocationUtilities.computeOperatorMemory(optionManager, ONE_GB, 100); assertEquals(40 * ONE_MB, opMinMem); } } From 1db65a271ef6574e1dd240669dc816fcd17740fd Mon Sep 17 00:00:00 2001 From: Sorabh Hamirwasia Date: Wed, 17 Apr 2019 14:28:59 -0700 Subject: [PATCH 3/7] DRILL-7191: RM blobs persistence in Zookeeper for Distributed RM. UUID support for DrillbitEndpoint RMState Blobs definition, serialization and deserialization, Zookeeper client support for transactions ZookeeperPersistentTransactional Store and RMStateBlobManager to do updates under lock Protect running and waiting queries map in WorkerBee --- .../org/apache/drill/exec/ExecConstants.java | 1 + .../drill/exec/coord/ClusterCoordinator.java | 19 +- .../coord/local/LocalClusterCoordinator.java | 52 ++- .../exec/coord/zk/ZKClusterCoordinator.java | 103 ++--- .../exec/coord/zk/ZKRegistrationHandle.java | 15 +- .../drill/exec/coord/zk/ZookeeperClient.java | 122 +++++- .../drill/exec/resourcemgr/NodeResources.java | 93 ++++- .../config/QueryQueueConfigImpl.java | 2 +- .../exception/QueueWaitTimeoutExpired.java | 28 ++ .../rmblobmgr/RMBlobStoreManager.java | 39 ++ .../RMConsistentBlobStoreManager.java | 354 ++++++++++++++++++ .../exception/LeaderChangeException.java | 28 ++ .../exception/RMBlobUpdateException.java | 28 ++ .../ResourceUnavailableException.java | 28 ++ .../rmblobmgr/rmblob/AbstractRMStateBlob.java | 35 ++ .../rmblobmgr/rmblob/ClusterStateBlob.java | 70 ++++ .../rmblob/ForemanQueueUsageBlob.java | 67 ++++ .../rmblob/ForemanResourceUsage.java | 107 ++++++ .../rmblobmgr/rmblob/QueueLeadershipBlob.java | 63 ++++ .../rmblobmgr/rmblob/RMStateBlob.java | 26 ++ .../exec/store/sys/PersistentStoreConfig.java | 15 +- .../exec/store/sys/PersistentStoreMode.java | 3 +- .../sys/store/ZookeeperPersistentStore.java | 21 +- ...ookeeperTransactionalPersistenceStore.java | 111 ++++++ .../ZookeeperPersistentStoreProvider.java | 43 ++- .../apache/drill/exec/work/WorkManager.java | 99 ++++- .../drill/exec/work/foreman/Foreman.java | 120 ++++-- .../foreman/rm/DefaultResourceManager.java | 39 +- .../rm/DistributedResourceManager.java | 251 +++++++++++-- .../foreman/rm/DynamicResourceManager.java | 5 + .../work/foreman/rm/QueryResourceManager.java | 51 ++- .../exec/work/foreman/rm/ResourceManager.java | 2 + .../foreman/rm/ThrottledResourceManager.java | 44 ++- .../src/main/resources/drill-module.conf | 6 +- .../exec/resourcemgr/RMBlobManagerTest.java | 171 +++++++++ 35 files changed, 2047 insertions(+), 214 deletions(-) create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/exception/QueueWaitTimeoutExpired.java create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/RMBlobStoreManager.java create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/RMConsistentBlobStoreManager.java create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/exception/LeaderChangeException.java create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/exception/RMBlobUpdateException.java create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/exception/ResourceUnavailableException.java create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/rmblob/AbstractRMStateBlob.java create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/rmblob/ClusterStateBlob.java create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/rmblob/ForemanQueueUsageBlob.java create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/rmblob/ForemanResourceUsage.java create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/rmblob/QueueLeadershipBlob.java create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/rmblob/RMStateBlob.java create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/store/ZookeeperTransactionalPersistenceStore.java create mode 100644 exec/java-exec/src/test/java/org/apache/drill/exec/resourcemgr/RMBlobManagerTest.java diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java index 24372ef5e40..3fac47c8286 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java @@ -691,6 +691,7 @@ private ExecConstants() { new OptionDescription("Indicates how long a query can wait in queue before the query fails. Range: 0-9223372036854775807")); // New Smart RM boot time configs + public static final String RM_WAIT_THREAD_INTERVAL = "exec.rm.wait_thread_interval"; public static final String RM_QUERY_TAGS_KEY = "exec.rm.queryTags"; public static final StringValidator RM_QUERY_TAGS_VALIDATOR = new StringValidator(RM_QUERY_TAGS_KEY, new OptionDescription("Allows user to set coma separated list of tags for all the queries submitted over a session")); 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 aa7f66a8a10..75654171003 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 @@ -17,16 +17,17 @@ */ package org.apache.drill.exec.coord; -import java.util.Collection; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; - import org.apache.drill.exec.coord.store.TransientStore; import org.apache.drill.exec.coord.store.TransientStoreConfig; import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint; import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.State; import org.apache.drill.exec.work.foreman.DrillbitStatusListener; +import java.util.Collection; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + /** * Pluggable interface built to manage cluster coordination. Allows Drillbit or DrillClient to register its capabilities * as well as understand other node's existence and capabilities. @@ -60,6 +61,10 @@ public abstract class ClusterCoordinator implements AutoCloseable { */ public abstract Collection getAvailableEndpoints(); + public Map getAvailableEndpointsUUID() { + throw new UnsupportedOperationException("Only supported by Zookeeper Cluster Coordinator outside YARN"); + } + /** * Get a collection of ONLINE drillbit endpoints by excluding the drillbits * that are in QUIESCENT state (drillbits that are shutting down). Primarily used by the planner @@ -70,6 +75,10 @@ public abstract class ClusterCoordinator implements AutoCloseable { public abstract Collection getOnlineEndPoints(); + public Map getOnlineEndpointsUUID() { + throw new UnsupportedOperationException("Only supported by Zookeeper Cluster Coordinator outside YARN"); + } + public abstract RegistrationHandle update(RegistrationHandle handle, State state); public interface RegistrationHandle { @@ -79,6 +88,8 @@ public interface RegistrationHandle { */ public abstract DrillbitEndpoint getEndPoint(); + public abstract String getId(); + public abstract void setEndPoint(DrillbitEndpoint endpoint); } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/coord/local/LocalClusterCoordinator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/coord/local/LocalClusterCoordinator.java index 16a7e9d35cb..8d2fedc19d3 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/coord/local/LocalClusterCoordinator.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/coord/local/LocalClusterCoordinator.java @@ -17,15 +17,6 @@ */ package org.apache.drill.exec.coord.local; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Map; -import java.util.UUID; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.Semaphore; -import java.util.concurrent.TimeUnit; - import org.apache.drill.exec.coord.ClusterCoordinator; import org.apache.drill.exec.coord.DistributedSemaphore; import org.apache.drill.exec.coord.store.CachingTransientStoreFactory; @@ -34,9 +25,17 @@ import org.apache.drill.exec.coord.store.TransientStoreFactory; 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.collect.Maps; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; + public class LocalClusterCoordinator extends ClusterCoordinator { private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(LocalClusterCoordinator.class); @@ -103,7 +102,15 @@ public RegistrationHandle update(RegistrationHandle handle, State state) { @Override public Collection getAvailableEndpoints() { - return endpoints.values(); + return getAvailableEndpointsUUID().values(); + } + + public Map getAvailableEndpointsUUID() { + Map availableEndpointsUUID = new HashMap<>(); + for (Map.Entry entry : endpoints.entrySet()) { + availableEndpointsUUID.put(entry.getKey().getId(), entry.getValue()); + } + return availableEndpointsUUID; } /** @@ -115,19 +122,27 @@ public Collection getAvailableEndpoints() { */ @Override public Collection getOnlineEndPoints() { - Collection runningEndPoints = new ArrayList<>(); - for (DrillbitEndpoint endpoint: endpoints.values()){ - if(isDrillbitInState(endpoint, State.ONLINE)) { - runningEndPoints.add(endpoint); + return getOnlineEndpointsUUID().keySet(); + } + + public Map getOnlineEndpointsUUID() { + Map onlineEndpointsUUID = new HashMap<>(); + for (Map.Entry entry : endpoints.entrySet()) { + if(isDrillbitInState(entry.getValue(), State.ONLINE)) { + onlineEndpointsUUID.put(entry.getValue(), entry.getKey().getId()); } } - return runningEndPoints; + return onlineEndpointsUUID; } private class Handle implements RegistrationHandle { private final UUID id = UUID.randomUUID(); private DrillbitEndpoint drillbitEndpoint; + private Handle(DrillbitEndpoint data) { + drillbitEndpoint = data; + } + /** * Get the drillbit endpoint associated with the registration handle * @return drillbit endpoint @@ -140,8 +155,9 @@ public void setEndPoint(DrillbitEndpoint endpoint) { this.drillbitEndpoint = endpoint; } - private Handle(DrillbitEndpoint data) { - drillbitEndpoint = data; + @Override + public String getId() { + return id.toString(); } @Override 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 74dd01cf9e9..3d276c05683 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 @@ -17,26 +17,11 @@ */ package org.apache.drill.exec.coord.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.ArrayList; -import java.util.Set; -import java.util.HashSet; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; -import java.util.regex.Matcher; -import java.util.regex.Pattern; - -import org.apache.curator.framework.imps.DefaultACLProvider; -import org.apache.drill.shaded.guava.com.google.common.base.Throwables; -import org.apache.commons.collections.keyvalue.MultiKey; import org.apache.curator.RetryPolicy; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; import org.apache.curator.framework.api.ACLProvider; +import org.apache.curator.framework.imps.DefaultACLProvider; import org.apache.curator.framework.state.ConnectionState; import org.apache.curator.framework.state.ConnectionStateListener; import org.apache.curator.retry.RetryNTimes; @@ -57,7 +42,21 @@ import org.apache.drill.exec.coord.store.TransientStoreFactory; 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.Function; +import org.apache.drill.shaded.guava.com.google.common.base.Throwables; + +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; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; /** * Manages cluster coordination utilizing zookeeper. * @@ -74,8 +73,8 @@ public class ZKClusterCoordinator extends ClusterCoordinator { private ServiceCache serviceCache; private DrillbitEndpoint endpoint; - // endpointsMap maps Multikey( comprises of endoint address and port) to Drillbit endpoints - private ConcurrentHashMap endpointsMap = new ConcurrentHashMap(); + // endpointsMap maps String UUID to Drillbit endpoints + private ConcurrentHashMap endpointsMap = new ConcurrentHashMap<>(); private static final Pattern ZK_COMPLEX_STRING = Pattern.compile("(^.*?)/(.*)/([^/]*)$"); public ZKClusterCoordinator(DrillConfig config, String connect) { @@ -237,7 +236,12 @@ public RegistrationHandle update(RegistrationHandle handle, State state) { @Override public Collection getAvailableEndpoints() { - return this.endpoints; + return getAvailableEndpointsUUID().values(); + } + + @Override + public Map getAvailableEndpointsUUID() { + return this.endpointsMap; } /* @@ -249,14 +253,19 @@ public Collection getAvailableEndpoints() { */ @Override public Collection getOnlineEndPoints() { - Collection runningEndPoints = new ArrayList<>(); - for (DrillbitEndpoint endpoint: endpoints){ - if(isDrillbitInState(endpoint, State.ONLINE)) { - runningEndPoints.add(endpoint); + return getOnlineEndpointsUUID().keySet(); + } + + @Override + public Map getOnlineEndpointsUUID() { + Map onlineEndpointsUUID = new HashMap<>(); + for (Map.Entry endpointEntry : endpointsMap.entrySet()) { + if (isDrillbitInState(endpointEntry.getValue(), State.ONLINE)) { + onlineEndpointsUUID.put(endpointEntry.getValue(), endpointEntry.getKey()); } } - logger.debug("Online endpoints in ZK are" + runningEndPoints.toString()); - return runningEndPoints; + logger.debug("Online endpoints in ZK are" + onlineEndpointsUUID.keySet().toString()); + return onlineEndpointsUUID; } @Override @@ -273,14 +282,11 @@ public TransientStore getOrCreateTransientStore(final TransientStoreConfi private synchronized void updateEndpoints() { try { // All active bits in the Zookeeper - Collection newDrillbitSet = - transform(discovery.queryForInstances(serviceName), - new Function, DrillbitEndpoint>() { - @Override - public DrillbitEndpoint apply(ServiceInstance input) { - return input.getPayload(); - } - }); + final Map activeEndpointsUUID = 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)); // set of newly dead bits : original bits - new set of active bits. Set unregisteredBits = new HashSet<>(); @@ -290,29 +296,32 @@ public DrillbitEndpoint apply(ServiceInstance input) { // 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 ( DrillbitEndpoint endpoint : newDrillbitSet) { - String endpointAddress = endpoint.getAddress(); - int endpointPort = endpoint.getUserPort(); - if (! endpointsMap.containsKey(new MultiKey(endpointAddress, endpointPort))) { - registeredBits.add(endpoint); - } - endpointsMap.put(new MultiKey(endpointAddress, endpointPort),endpoint); + for (Map.Entry endpointToUUID : activeEndpointsUUID.entrySet()) { + endpointsMap.put(endpointToUUID.getKey(), endpointToUUID.getValue()); } + // Remove all the endpoints that are newly dead - for ( MultiKey key: endpointsMap.keySet()) { - if(!newDrillbitSet.contains(endpointsMap.get(key))) { - unregisteredBits.add(endpointsMap.get(key)); - endpointsMap.remove(key); + for ( String bitUUID: endpointsMap.keySet()) { + if (!activeEndpointsUUID.containsKey(bitUUID)) { + final DrillbitEndpoint unregisteredBit = endpointsMap.get(bitUUID); + unregisteredBits.add(unregisteredBit); + + if (UUIDtoEndpoints.containsKey(unregisteredBit)) { + logger.info("Drillbit registered again with different UUID. [Details: Address: {}, UserPort: {}," + + " PreviousUUID: {}, CurrentUUID: {}", unregisteredBit.getAddress(), unregisteredBit.getUserPort(), + bitUUID, UUIDtoEndpoints.get(unregisteredBit)); + } + endpointsMap.remove(bitUUID); } } endpoints = endpointsMap.values(); if (logger.isDebugEnabled()) { StringBuilder builder = new StringBuilder(); builder.append("Active drillbit set changed. Now includes "); - builder.append(newDrillbitSet.size()); + builder.append(activeEndpointsUUID.size()); builder.append(" total bits. New active drillbits:\n"); builder.append("Address | User Port | Control Port | Data Port | Version | State\n"); - for (DrillbitEndpoint bit: newDrillbitSet) { + for (DrillbitEndpoint bit: activeEndpointsUUID.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/coord/zk/ZKRegistrationHandle.java b/exec/java-exec/src/main/java/org/apache/drill/exec/coord/zk/ZKRegistrationHandle.java index 4f3730fbd0d..0d182cb5010 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/coord/zk/ZKRegistrationHandle.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/coord/zk/ZKRegistrationHandle.java @@ -26,19 +26,24 @@ public class ZKRegistrationHandle implements RegistrationHandle { public final String id; public DrillbitEndpoint endpoint; + public ZKRegistrationHandle(String id, DrillbitEndpoint endpoint) { + super(); + this.id = id; + this.endpoint = endpoint; + } + public DrillbitEndpoint getEndPoint() { return endpoint; } + public String getId() { + return id; + } + @Override public void setEndPoint(DrillbitEndpoint endpoint) { this.endpoint = endpoint; } - public ZKRegistrationHandle(String id, DrillbitEndpoint endpoint) { - super(); - this.id = id; - this.endpoint = endpoint; - } } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/coord/zk/ZookeeperClient.java b/exec/java-exec/src/main/java/org/apache/drill/exec/coord/zk/ZookeeperClient.java index fee607d2ab2..89a37d94265 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/coord/zk/ZookeeperClient.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/coord/zk/ZookeeperClient.java @@ -17,27 +17,30 @@ */ package org.apache.drill.exec.coord.zk; -import java.util.Iterator; -import java.util.Map; - -import javax.annotation.Nullable; - -import org.apache.drill.shaded.guava.com.google.common.base.Function; -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.Iterables; import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.api.transaction.CuratorTransaction; +import org.apache.curator.framework.api.transaction.CuratorTransactionFinal; import org.apache.curator.framework.recipes.cache.ChildData; import org.apache.curator.framework.recipes.cache.PathChildrenCache; import org.apache.drill.common.collections.ImmutableEntry; import org.apache.drill.common.exceptions.DrillRuntimeException; import org.apache.drill.exec.exception.VersionMismatchException; import org.apache.drill.exec.store.sys.store.DataChangeVersion; +import org.apache.drill.shaded.guava.com.google.common.base.Function; +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.Iterables; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.NodeExistsException; import org.apache.zookeeper.data.Stat; +import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + /** * A namespace aware Zookeeper client. * @@ -227,13 +230,33 @@ public void create(final String path) { final String target = PathUtils.join(root, path); try { - curator.create().withMode(mode).forPath(target); + curator.create().creatingParentsIfNeeded().withMode(mode).forPath(target); getCache().rebuildNode(target); } catch (final Exception e) { throw new DrillRuntimeException("unable to put ", e); } } + public void createAsTransaction(List paths) { + Preconditions.checkNotNull(paths, "no paths provided to create"); + CuratorTransaction transaction = curator.inTransaction(); + List targetPaths = new ArrayList<>(); + try { + for (String path : paths) { + final String target = PathUtils.join(root, path); + targetPaths.add(target); + transaction = transaction.create().withMode(mode).forPath(target).and(); + } + ((CuratorTransactionFinal)transaction).commit(); + + for (String target : targetPaths) { + getCache().rebuildNode(target); + } + } catch (Exception ex) { + throw new DrillRuntimeException("Failed to create paths in transactional manner", ex); + } + } + /** * Puts the given byte sequence into the given path. * @@ -273,7 +296,7 @@ public void put(final String path, final byte[] data, DataChangeVersion version) boolean hasNode = hasPath(path, true); if (!hasNode) { try { - curator.create().withMode(mode).forPath(target, data); + curator.create().creatingParentsIfNeeded().withMode(mode).forPath(target, data); } catch (NodeExistsException e) { // Handle race conditions since Drill is distributed and other // drillbits may have just created the node. This assumes that we do want to @@ -301,6 +324,54 @@ public void put(final String path, final byte[] data, DataChangeVersion version) } } + public void putAsTransaction(Map pathsWithData) { + putAsTransaction(pathsWithData, null); + } + + /** + * Puts the given sets of blob and their data's in a transactional manner. It expects all the blob path to exist + * before calling this api. + * @param pathsWithData - map of blob paths to update and the final data + * @param version - version holder + */ + public void putAsTransaction(Map pathsWithData, DataChangeVersion version) { + Preconditions.checkNotNull(pathsWithData, "paths and their data to write as transaction is missing"); + List targetPaths = new ArrayList<>(); + CuratorTransaction transaction = curator.inTransaction(); + long totalDataBytes = 0; + + try { + for (Map.Entry entry : pathsWithData.entrySet()) { + final String target = PathUtils.join(root, entry.getKey()); + + if (version != null) { + transaction = transaction.setData().withVersion(version.getVersion()).forPath(target, entry.getValue()).and(); + } else { + transaction = transaction.setData().forPath(target, entry.getValue()).and(); + } + targetPaths.add(target); + totalDataBytes += entry.getValue().length; + } + + // If total set operator payload is greater than 1MB then curator set operation will fail + if (totalDataBytes >= 1_048_576) { + throw new UnsupportedOperationException("Curator doesn't support transactional put of more than 1 MB"); + } + + // if commit fails then an exception will be thrown + ((CuratorTransactionFinal)transaction).commit(); + // if successful then rebuild the cache + for (String target : targetPaths) { + getCache().rebuildNode(target); + } + } catch (KeeperException.BadVersionException ex) { + throw new VersionMismatchException("Failed to put data to blobs as a single transaction because of mismatch " + + "in version", version.getVersion(), ex); + } catch (Exception ex) { + throw new DrillRuntimeException("Failed to write data to blobs as a single transaction", ex); + } + } + /** * Puts the given byte sequence into the given path if path is does not exist. * @@ -315,7 +386,7 @@ public byte[] putIfAbsent(final String path, final byte[] data) { final String target = PathUtils.join(root, path); try { try { - curator.create().withMode(mode).forPath(target, data); + curator.create().creatingParentsIfNeeded().withMode(mode).forPath(target, data); getCache().rebuildNode(target); return null; } catch (NodeExistsException e) { @@ -344,6 +415,33 @@ public void delete(final String path) { } } + /** + * Deletes all the blobs residing at list of given paths as a single transaction + * + * @param pathList target paths to delete + */ + public void deleteAsTransaction(List pathList) { + Preconditions.checkNotNull(pathList, "List of paths to delete as transaction is required"); + final List targetPaths = new ArrayList<>(); + CuratorTransaction transaction = curator.inTransaction(); + + try { + for (String path : pathList) { + final String target = PathUtils.join(root, path); + transaction = transaction.delete().forPath(target).and(); + targetPaths.add(target); + } + ((CuratorTransactionFinal)transaction).commit(); + + for (String target : targetPaths) { + getCache().rebuildNode(target); + } + } catch (Exception ex) { + throw new DrillRuntimeException(String.format("unable to delete blobs as transaction at %s", + String.join(",", targetPaths)), ex); + } + } + /** * Returns an iterator of (key, value) pairs residing under {@link #getRoot() root} path. */ 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 485702c8f43..7b632e6ac49 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 @@ -17,6 +17,18 @@ */ package org.apache.drill.exec.resourcemgr; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.deser.std.StdDeserializer; + +import java.io.IOException; +import java.util.Map; + /** * Provides resources for a node in cluster. Currently it is used to support only 2 kind of resources: *

    @@ -29,28 +41,39 @@ public class NodeResources { private final int version; - private final long memoryInBytes; + private long memoryInBytes; - private final int numVirtualCpu; + private int numVirtualCpu; + + private static final int CURRENT_VERSION = 1; public NodeResources(long memoryInBytes, int numVirtualCpu) { + this(CURRENT_VERSION, memoryInBytes, numVirtualCpu); + } + + @JsonCreator + public NodeResources(@JsonProperty("version") int version, + @JsonProperty("memoryInBytes") long memoryInBytes, + @JsonProperty("numVirtualCpu") int numVirtualCpu) { + this.version = version; this.memoryInBytes = memoryInBytes; this.numVirtualCpu = numVirtualCpu; - this.version = 1; } public NodeResources(long memoryInBytes, int numPhysicalCpu, int vFactor) { - this(memoryInBytes, numPhysicalCpu * vFactor); + this(CURRENT_VERSION, memoryInBytes, numPhysicalCpu * vFactor); } public long getMemoryInBytes() { return memoryInBytes; } + @JsonIgnore public long getMemoryInMB() { return Math.round((memoryInBytes / 1024L) / 1024L); } + @JsonIgnore public long getMemoryInGB() { return Math.round(getMemoryInMB() / 1024L); } @@ -59,8 +82,70 @@ public int getNumVirtualCpu() { return numVirtualCpu; } + public int getVersion() { + return version; + } + + public void setMemoryInBytes(long memoryInBytes) { + this.memoryInBytes = memoryInBytes; + } + + public void setNumVirtualCpu(int numVCpu) { + this.numVirtualCpu = numVCpu; + } + + public void add(NodeResources other) { + if (other == null) { + return; + } + this.numVirtualCpu += other.getNumVirtualCpu(); + this.memoryInBytes += other.getMemoryInBytes(); + } + + public static Map merge(Map to, + Map from) { + to.entrySet().stream().forEach((toEntry) -> toEntry.getValue().add(from.get(toEntry.getKey()))); + return to; + } + @Override public String toString() { return "{ Version: " + version + ", MemoryInBytes: " + memoryInBytes + ", VirtualCPU: " + numVirtualCpu + " }"; } + + @Override + public int hashCode() { + int result = 31 ^ Integer.hashCode(version); + result = result ^ Integer.hashCode(numVirtualCpu); + result = result ^ Long.hashCode(memoryInBytes); + return result; + } + + @Override + public boolean equals(Object obj) { + if (obj == null || (obj.getClass() != this.getClass())) { + return false; + } + + if (this == obj) { + return true; + } + NodeResources other = (NodeResources) obj; + return this.version == other.getVersion() && this.numVirtualCpu == other.getNumVirtualCpu() && + this.memoryInBytes == other.getMemoryInBytes(); + } + + public static class NodeResourcesDe extends StdDeserializer { + + private static final ObjectMapper mapper = new ObjectMapper(); + + public NodeResourcesDe() { + super(NodeResources.class); + } + + @Override + public NodeResources deserialize(JsonParser p, DeserializationContext ctxt) throws IOException, JsonProcessingException { + return mapper.readValue(p, NodeResources.class); + } + } } 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 948d1b943b5..6108f039306 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 @@ -47,7 +47,7 @@ public class QueryQueueConfigImpl implements QueryQueueConfig { private static final String MAX_QUERY_MEMORY_PER_NODE_FORMAT = "([0-9]+)\\s*([kKmMgG]?)\\s*$"; // Required queue configurations in MAX_QUERY_MEMORY_PER_NODE_FORMAT pattern - private static final String MAX_QUERY_MEMORY_PER_NODE_KEY = "max_query_memory_per_node"; + public static final String MAX_QUERY_MEMORY_PER_NODE_KEY = "max_query_memory_per_node"; private final String queueUUID; diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/exception/QueueWaitTimeoutExpired.java b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/exception/QueueWaitTimeoutExpired.java new file mode 100644 index 00000000000..f2354869883 --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/exception/QueueWaitTimeoutExpired.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.drill.exec.resourcemgr.exception; + +public class QueueWaitTimeoutExpired extends Exception { + public QueueWaitTimeoutExpired(String message) { + super(message); + } + + public QueueWaitTimeoutExpired(String message, Exception ex) { + super(message, ex); + } +} 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 new file mode 100644 index 00000000000..2cfa654e851 --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/RMBlobStoreManager.java @@ -0,0 +1,39 @@ +/* + * 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.rmblobmgr; + +import org.apache.drill.exec.resourcemgr.NodeResources; +import org.apache.drill.exec.resourcemgr.config.QueryQueueConfig; + +import java.util.Map; + +/** + * Interface that defines implementation of StoreManager for RMStateBlobs. BlobManager is the responsible for + * initialization of blobs and all the reads and writes to the blobs. It exposes mainly 2 apis which are used when blobs + * needs to be updated. One is to update the blobs to reflect that resources are reserved for this query whereas + * other is used to update the blobs to free up the resources. There can be different implementations of the store + * manager where one can manage the blobs lazily whereas other will manage it in strongly consistent manner. Current + * implementation {@link RMConsistentBlobStoreManager} is using the later approach. + */ +public interface RMBlobStoreManager { + void reserveResources(Map queryResourceAssignment, QueryQueueConfig selectedQueue, + String leaderId, String queryId, String foremanNode) throws Exception; + + void freeResources(Map queryResourceAssignment, QueryQueueConfig selectedQueue, + String leaderId, String queryId, String foremanNode) 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 new file mode 100644 index 00000000000..33e719df9fb --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/RMConsistentBlobStoreManager.java @@ -0,0 +1,354 @@ +/* + * 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.rmblobmgr; + +import avro.shaded.com.google.common.annotations.VisibleForTesting; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.SerializationFeature; +import com.fasterxml.jackson.databind.module.SimpleModule; +import org.apache.curator.framework.recipes.locks.InterProcessMutex; +import org.apache.drill.common.scanner.persistence.ScanResult; +import org.apache.drill.exec.coord.zk.ZKClusterCoordinator; +import org.apache.drill.exec.exception.StoreException; +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.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; +import org.apache.drill.exec.resourcemgr.rmblobmgr.rmblob.ForemanResourceUsage.ForemanResourceUsageDe; +import org.apache.drill.exec.resourcemgr.rmblobmgr.rmblob.QueueLeadershipBlob; +import org.apache.drill.exec.resourcemgr.rmblobmgr.rmblob.RMStateBlob; +import org.apache.drill.exec.server.DrillbitContext; +import org.apache.drill.exec.store.sys.PersistentStoreConfig; +import org.apache.drill.exec.store.sys.store.ZookeeperTransactionalPersistenceStore; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +/** + * RM state blobs manager which does all the update to the blobs under a global lock and in transactional manner. + * Since the blobs are updated by multiple Drillbit at same time to maintain the strongly consistent information in + * these blobs it uses a global lock shared across all the Drillbits. + */ +public class RMConsistentBlobStoreManager implements RMBlobStoreManager { + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RMConsistentBlobStoreManager.class); + + private static final String RM_BLOBS_ROOT = "rm/blobs"; + + private static final String RM_LOCK_ROOT = "/rm/locks"; + + private static final String RM_BLOB_GLOBAL_LOCK_NAME = "/rm_blob_lock"; + + private static final String RM_BLOB_SER_DE_NAME = "RMStateBlobSerDeModules"; + + public static final int RM_STATE_BLOB_VERSION = 1; + + private static final int MAX_ACQUIRE_RETRY = 3; + + private final ZookeeperTransactionalPersistenceStore rmBlobStore; + + private final InterProcessMutex globalBlobMutex; + + private final DrillbitContext context; + + private final ObjectMapper serDeMapper; + + private final Map rmStateBlobs; + + private final StringBuilder exceptionStringBuilder = new StringBuilder(); + + public RMConsistentBlobStoreManager(DrillbitContext context, Collection leafQueues) throws + StoreException { + try { + this.context = context; + this.serDeMapper = initializeMapper(context.getClasspathScan()); + this.rmBlobStore = (ZookeeperTransactionalPersistenceStore) context.getStoreProvider() + .getOrCreateStore(PersistentStoreConfig.newJacksonBuilder(serDeMapper, RMStateBlob.class) + .name(RM_BLOBS_ROOT) + .persistWithTransaction() + .build()); + this.globalBlobMutex = new InterProcessMutex(((ZKClusterCoordinator) context.getClusterCoordinator()).getCurator(), + RM_LOCK_ROOT + RM_BLOB_GLOBAL_LOCK_NAME); + this.rmStateBlobs = new HashMap<>(); + initializeBlobs(leafQueues); + } catch (StoreException ex) { + throw ex; + } catch (Exception ex) { + throw new StoreException("Failed to initialize RM State Blobs", ex); + } + } + + private Collection> getAllBlobSubTypes(ScanResult classpathScan) { + return new ArrayList<>(classpathScan.getImplementations(RMStateBlob.class)); + } + + private ObjectMapper initializeMapper(ScanResult scanResult) { + final ObjectMapper mapper = new ObjectMapper(); + mapper.enable(SerializationFeature.INDENT_OUTPUT); + mapper.configure(JsonParser.Feature.ALLOW_UNQUOTED_FIELD_NAMES, true); + mapper.configure(JsonGenerator.Feature.QUOTE_FIELD_NAMES, true); + mapper.configure(JsonParser.Feature.ALLOW_COMMENTS, true); + mapper.registerSubtypes(getAllBlobSubTypes(scanResult)); + + final SimpleModule deserModule = new SimpleModule(RM_BLOB_SER_DE_NAME) + .addDeserializer(NodeResources.class, new NodeResourcesDe()) + .addDeserializer(ForemanResourceUsage.class, new ForemanResourceUsageDe()); + mapper.registerModule(deserModule); + return mapper; + } + + private void initializeBlobs(Collection leafQueues) throws Exception { + // acquire the global lock and ensure that all the blobs are created with empty data + int acquireTry = 1; + do { + try { + globalBlobMutex.acquire(); + break; + } catch (Exception ex) { + ++acquireTry; + } + } while (acquireTry <= MAX_ACQUIRE_RETRY); + + // if the lock is not acquired then just return as some other Drillbit can do it + // but there can be issues when none of the Drillbit is able to perform this operation + if (!globalBlobMutex.isAcquiredInThisProcess()) { + logger.warn("Failed to acquire global rm blobs lock to do blob initialization. Expectation is some other " + + "Drillbit should be able to do it"); + return; + } + + try { + logger.info("Acquired global rm blobs lock to do blob initialization"); + // 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<>())); + + // This ForemanResourceUsage blob needs to be per queue + final ForemanQueueUsageBlob queueUsageBlob = new ForemanQueueUsageBlob(RM_STATE_BLOB_VERSION, new HashMap<>()); + for (QueryQueueConfig queueConfig : leafQueues) { + final String blobName = ForemanQueueUsageBlob.NAME + "_" + queueConfig.getQueueName(); + rmStateBlobs.put(blobName, queueUsageBlob); + } + + for (Map.Entry stateBlob : rmStateBlobs.entrySet()) { + if (!rmBlobStore.putIfAbsent(stateBlob.getKey(), stateBlob.getValue())) { + logger.info("Blob {} was already initialized", stateBlob.getKey()); + } + } + } catch (Exception ex) { + // consume the exception during blob initialization since we are expecting some other Drillbit can do that + // successfully. If not then there will be failure in cluster during actual blob update + logger.error("Failed to initialize one or more blob with empty data, but consuming this exception since " + + "expectation is that some other Drillbit should be able to perform this step"); + } finally { + // throwing exception on release since it indicates mutex is in bad state + globalBlobMutex.release(); + } + } + + @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() + .collect(Collectors.toMap(Map.Entry::getKey, + (x) -> new NodeResources(x.getValue().getVersion(), + -x.getValue().getMemoryInBytes(), + -x.getValue().getNumVirtualCpu()))); + acquireLockAndUpdate(resourcesMap, selectedQueue, leaderId, queryId, foremanNode); + } + + @Override + public void freeResources(Map queryResourceAssignment, + QueryQueueConfig selectedQueue, String leaderId, + String queryId, String foremanNode) throws Exception { + acquireLockAndUpdate(queryResourceAssignment, selectedQueue, leaderId, queryId, foremanNode); + } + + private void updateBlobs(Map resourcesMap, QueryQueueConfig selectedQueue, + String leaderId, String queryId, String foremanNode) throws Exception { + + exceptionStringBuilder.append("QueryId: ").append(queryId) + .append(", ForemanBit: ").append(foremanNode) + .append(", QueueName: ").append(selectedQueue.getQueueName()) + .append(", Admitted Leader: ").append(leaderId); + + // get all the required blobs data as a transaction. Note: We won't use getAll since we have queue specific + // blobs too and the update will only for selected queue. So we should get data one blob at a time + final List blobsToGet = new ArrayList<>(); + final String queueBlobName = ForemanQueueUsageBlob.NAME + "_" + selectedQueue.getQueueName(); + blobsToGet.add(ClusterStateBlob.NAME); + blobsToGet.add(QueueLeadershipBlob.NAME); + blobsToGet.add(queueBlobName); + + final Map rmBlobs = rmBlobStore.getAllOrNone(blobsToGet); + if (rmBlobs == null) { + throw new RMBlobUpdateException(String.format("Failed to get one or more blob while update. [Details: %s]", + exceptionStringBuilder.toString())); + } else { + rmStateBlobs.putAll(rmBlobs); + } + + // Check if the leader admitting the query is still leader of the queue + final 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())); + } + // Remove leadership blob from cache + rmStateBlobs.remove(QueueLeadershipBlob.NAME); + + // Cluster state blob + final ClusterStateBlob currentClusterBlob = (ClusterStateBlob)rmStateBlobs.get(ClusterStateBlob.NAME); + final Map currentClusterState = currentClusterBlob.getClusterState(); + + // ForemanResourceUsage blob + final ForemanQueueUsageBlob resourceUsageBlob = (ForemanQueueUsageBlob)rmStateBlobs.get(queueBlobName); + final Map allForemanUsage = resourceUsageBlob.getAllForemanInfo(); + final ForemanResourceUsage currentUsage = allForemanUsage.get(foremanNode); + final Map usageMapAcrossDrillbits = currentUsage.getForemanUsage(); + int currentRunningCount = currentUsage.getRunningCount(); + + for (Map.Entry nodeToUpdate : resourcesMap.entrySet()) { + final String bitUUID = nodeToUpdate.getKey(); + final NodeResources bitResourcesToReserve = nodeToUpdate.getValue(); + + 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())); + } + 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, + currentAvailableMemory, memoryToReserve, exceptionStringBuilder.toString())); + } + // Update local ClusterState + bitAvailableResources.setMemoryInBytes(currentAvailableMemory - memoryToReserve); + currentClusterState.put(bitUUID, bitAvailableResources); + + // Update local ForemanResourceUsage for foremanNode with this query resource ask + final NodeResources currentState = usageMapAcrossDrillbits.get(bitUUID); + long availableMemory = currentState.getMemoryInBytes(); + currentState.setMemoryInBytes(availableMemory - memoryToReserve); + usageMapAcrossDrillbits.put(bitUUID, currentState); + } + + // update the local ClusterStateBlob with new information + currentClusterBlob.setClusterState(currentClusterState); + + // update the local ForemanQueueUsageBlob with final ForemanResourceUsage + currentUsage.setRunningCount(currentRunningCount + 1); + currentUsage.setForemanUsage(usageMapAcrossDrillbits); + allForemanUsage.put(foremanNode, currentUsage); + resourceUsageBlob.setAllForemanInfo(allForemanUsage); + + // Update local blob cache + rmStateBlobs.put(ClusterStateBlob.NAME, currentClusterBlob); + rmStateBlobs.put(queueBlobName, resourceUsageBlob); + + // Persist the new blobs to Zookeeper + if (!writeAllRMBlobs(rmStateBlobs)) { + logger.error("Failed to update the cluster state blob and queue blob for queue {} in a transaction", + selectedQueue.getQueueName()); + throw new RMBlobUpdateException(String.format("Failed to update the cluster state blob and queue blob in a " + + "transaction. [Details: %s]", exceptionStringBuilder.toString())); + } + logger.debug("Successfully updated the blobs in a transaction. [Details: %s]", exceptionStringBuilder.toString()); + + // Reset the exceptionStringBuilder for next event + exceptionStringBuilder.delete(0, exceptionStringBuilder.length()); + } + + private void acquireLockAndUpdate(Map queryResourceAssignment, QueryQueueConfig selectedQueue, + String leaderId, String queryId, String foremanNode) throws Exception { + try { + globalBlobMutex.acquire(); + } catch (Exception ex) { + logger.error("Failed on acquiring the global mutex while updating the RM blobs during update of resources"); + throw ex; + } + + try { + updateBlobs(queryResourceAssignment, selectedQueue, leaderId, queryId, foremanNode); + } catch (Exception ex) { + logger.error("Failed to update the blobs", ex); + throw 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 the RM blobs during update of resources", + ex); + // don't throw this release exception instead throw the original exception if any. Since release exception + // should not matter much + } + } + } + } + + @VisibleForTesting + public Iterator> readAllRMBlobs() { + return rmBlobStore.getAll(); + } + + @VisibleForTesting + public boolean writeAllRMBlobs(Map rmStateBlobs) { + return rmBlobStore.putAsTransaction(rmStateBlobs); + } + + @VisibleForTesting + public Map serializePassedInBlob(Map inputBlobs) throws Exception { + Map serializedBlobs = new HashMap<>(); + for (Map.Entry blobEntry : inputBlobs.entrySet()) { + serializedBlobs.put(blobEntry.getKey(), serDeMapper.writeValueAsString(blobEntry.getValue())); + } + return serializedBlobs; + } + + @VisibleForTesting + public Map deserializeRMStateBlobs(Map blobsInfo) throws Exception { + Map deserializedBlobs = new HashMap<>(); + for (Map.Entry blobEntry : blobsInfo.entrySet()) { + deserializedBlobs.put(blobEntry.getKey(), serDeMapper.readValue(blobEntry.getValue(), + RMStateBlob.class)); + } + return deserializedBlobs; + } +} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/exception/LeaderChangeException.java b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/exception/LeaderChangeException.java new file mode 100644 index 00000000000..a41191e699a --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/exception/LeaderChangeException.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.drill.exec.resourcemgr.rmblobmgr.exception; + +public class LeaderChangeException extends Exception { + public LeaderChangeException(String message) { + super(message); + } + + public LeaderChangeException(String message, Exception ex) { + super(message, ex); + } +} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/exception/RMBlobUpdateException.java b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/exception/RMBlobUpdateException.java new file mode 100644 index 00000000000..61e7038dbc0 --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/exception/RMBlobUpdateException.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.drill.exec.resourcemgr.rmblobmgr.exception; + +public class RMBlobUpdateException extends Exception { + public RMBlobUpdateException(String message) { + super(message); + } + + public RMBlobUpdateException(String message, Exception ex) { + super(message, ex); + } +} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/exception/ResourceUnavailableException.java b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/exception/ResourceUnavailableException.java new file mode 100644 index 00000000000..474d7046170 --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/exception/ResourceUnavailableException.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.drill.exec.resourcemgr.rmblobmgr.exception; + +public class ResourceUnavailableException extends Exception { + public ResourceUnavailableException(String message) { + super(message); + } + + public ResourceUnavailableException(String message, Exception ex) { + super(message, ex); + } +} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/rmblob/AbstractRMStateBlob.java b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/rmblob/AbstractRMStateBlob.java new file mode 100644 index 00000000000..7894151983d --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/rmblob/AbstractRMStateBlob.java @@ -0,0 +1,35 @@ +/* + * 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.rmblobmgr.rmblob; + +import com.fasterxml.jackson.annotation.JsonCreator; + +public abstract class AbstractRMStateBlob implements RMStateBlob { + + protected int version; + + @JsonCreator + AbstractRMStateBlob(int version) { + this.version = version; + } + + @Override + public int getVersion() { + return version; + } +} 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 new file mode 100644 index 00000000000..7faefcf4bf4 --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/rmblob/ClusterStateBlob.java @@ -0,0 +1,70 @@ +/* + * 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.rmblobmgr.rmblob; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.fasterxml.jackson.databind.annotation.JsonDeserialize; +import org.apache.drill.exec.resourcemgr.NodeResources; + +import java.util.Map; +@JsonTypeName(ClusterStateBlob.NAME) +public class ClusterStateBlob extends AbstractRMStateBlob { + public static final String NAME = "cluster_usage"; + + @JsonDeserialize(contentUsing = NodeResources.NodeResourcesDe.class) + private Map clusterState; + + @JsonCreator + public ClusterStateBlob(@JsonProperty("version") int version, + @JsonProperty("clusterState") Map clusterState) { + super(version); + this.clusterState = clusterState; + } + + public Map getClusterState() { + return clusterState; + } + + public void setClusterState(Map clusterState) { + this.clusterState = clusterState; + } + + @Override + public int hashCode() { + int result = 31; + result = result ^ Integer.hashCode(version); + result = result ^ clusterState.hashCode(); + return result; + } + + @Override + public boolean equals(Object obj) { + if (obj == null || (obj.getClass() != this.getClass())) { + return false; + } + + if (this == obj) { + return true; + } + + ClusterStateBlob other = (ClusterStateBlob) obj; + return this.version == other.getVersion() && this.clusterState.equals(other.getClusterState()); + } +} \ No newline at end of file diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/rmblob/ForemanQueueUsageBlob.java b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/rmblob/ForemanQueueUsageBlob.java new file mode 100644 index 00000000000..56ca2f4cbda --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/rmblob/ForemanQueueUsageBlob.java @@ -0,0 +1,67 @@ +/* + * 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.rmblobmgr.rmblob; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; + +import java.util.Map; +@JsonTypeName(ForemanQueueUsageBlob.NAME) +public class ForemanQueueUsageBlob extends AbstractRMStateBlob { + public static final String NAME = "foreman_queue_usage"; + + private Map allForemanInfo; + + @JsonCreator + public ForemanQueueUsageBlob(@JsonProperty("version") int version, + @JsonProperty("allForemanInfo") Map allForemanInfo) { + super(version); + this.allForemanInfo = allForemanInfo; + } + + public Map getAllForemanInfo() { + return allForemanInfo; + } + + public void setAllForemanInfo(Map allForemanInfo) { + this.allForemanInfo = allForemanInfo; + } + + @Override + public int hashCode() { + int result = 31; + result = result ^ Integer.hashCode(version); + result = result ^ allForemanInfo.hashCode(); + return result; + } + + @Override + public boolean equals(Object obj) { + if (obj == null || (obj.getClass() != this.getClass())) { + return false; + } + + if (this == obj) { + return true; + } + + ForemanQueueUsageBlob other = (ForemanQueueUsageBlob) obj; + return this.version == other.getVersion() && this.allForemanInfo.equals(other.getAllForemanInfo()); + } +} \ No newline at end of file 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 new file mode 100644 index 00000000000..de6f1fed80e --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/rmblob/ForemanResourceUsage.java @@ -0,0 +1,107 @@ +/* + * 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.rmblobmgr.rmblob; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.core.JsonProcessingException; +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.resourcemgr.NodeResources; + +import java.io.IOException; +import java.util.Map; + +public class ForemanResourceUsage { + + private int version; + + private Map foremanUsage; + + private int runningCount; + + @JsonCreator + public ForemanResourceUsage(@JsonProperty("version") int version, + @JsonProperty("foremanUsage") Map foremanUsage, + @JsonProperty("runningCount") int runningCount) { + this.version = version; + this.foremanUsage = foremanUsage; + this.runningCount = runningCount; + } + + public int getVersion() { + return version; + } + + public Map getForemanUsage() { + return foremanUsage; + } + + public int getRunningCount() { + return runningCount; + } + + public void setRunningCount(int runningCount) { + this.runningCount = runningCount; + } + + public void setForemanUsage(Map foremanUsage) { + this.foremanUsage = foremanUsage; + } + + @Override + public int hashCode() { + int result = 31; + result = result ^ Integer.hashCode(version); + result = result ^ Integer.hashCode(runningCount); + result = result ^ foremanUsage.hashCode(); + return result; + } + + @Override + public boolean equals(Object obj) { + if (obj == null || (obj.getClass() != this.getClass())) { + return false; + } + + if (this == obj) { + return true; + } + + ForemanResourceUsage other = (ForemanResourceUsage) obj; + return this.version == other.getVersion() && this.runningCount == other.getRunningCount() && + this.foremanUsage.equals(other.getForemanUsage()); + } + + public static class ForemanResourceUsageDe extends StdDeserializer { + + private static final ObjectMapper mapper = new ObjectMapper(); + + public ForemanResourceUsageDe() { + super(ForemanResourceUsage.class); + } + + @Override + public ForemanResourceUsage deserialize(JsonParser p, DeserializationContext ctxt) + throws IOException, JsonProcessingException { + return mapper.readValue(p, ForemanResourceUsage.class); + } + } +} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/rmblob/QueueLeadershipBlob.java b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/rmblob/QueueLeadershipBlob.java new file mode 100644 index 00000000000..d27240f47e2 --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/rmblob/QueueLeadershipBlob.java @@ -0,0 +1,63 @@ +/* + * 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.rmblobmgr.rmblob; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; + +import java.util.Map; +@JsonTypeName(QueueLeadershipBlob.NAME) +public class QueueLeadershipBlob extends AbstractRMStateBlob { + public static final String NAME = "queue_leaders"; + + private Map queueLeaders; + + @JsonCreator + public QueueLeadershipBlob(@JsonProperty("version") int version, + @JsonProperty("queueLeaders") Map queueLeaders) { + super(version); + this.queueLeaders = queueLeaders; + } + + public Map getQueueLeaders() { + return queueLeaders; + } + + @Override + public int hashCode() { + int result = 31; + result = result ^ Integer.hashCode(version); + result = result ^ queueLeaders.hashCode(); + return result; + } + + @Override + public boolean equals(Object obj) { + if (obj == null || (obj.getClass() != this.getClass())) { + return false; + } + + if (this == obj) { + return true; + } + + QueueLeadershipBlob other = (QueueLeadershipBlob) obj; + return this.version == other.getVersion() && this.queueLeaders.equals(other.getQueueLeaders()); + } +} \ No newline at end of file diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/rmblob/RMStateBlob.java b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/rmblob/RMStateBlob.java new file mode 100644 index 00000000000..16945080fdc --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/resourcemgr/rmblobmgr/rmblob/RMStateBlob.java @@ -0,0 +1,26 @@ +/* + * 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.rmblobmgr.rmblob; + +import com.fasterxml.jackson.annotation.JsonTypeInfo; + +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property="type") +public interface RMStateBlob { + + int getVersion(); +} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/PersistentStoreConfig.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/PersistentStoreConfig.java index 532033a0dbc..29c91dfbb1a 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/PersistentStoreConfig.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/PersistentStoreConfig.java @@ -103,14 +103,21 @@ public StoreConfigBuilder name(String name) { return this; } - public StoreConfigBuilder persist(){ - this.mode = PersistentStoreMode.PERSISTENT; + public StoreConfigBuilder mode(PersistentStoreMode mode) { + this.mode = mode; return this; } + public StoreConfigBuilder persist(){ + return mode(PersistentStoreMode.PERSISTENT); + } + public StoreConfigBuilder blob(){ - this.mode = PersistentStoreMode.BLOB_PERSISTENT; - return this; + return mode(PersistentStoreMode.BLOB_PERSISTENT); + } + + public StoreConfigBuilder persistWithTransaction(){ + return mode(PersistentStoreMode.PERSISTENT_TRANSACTION); } public StoreConfigBuilder setCapacity(int capacity) { diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/PersistentStoreMode.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/PersistentStoreMode.java index e099315dfd3..8ea232b9bee 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/PersistentStoreMode.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/PersistentStoreMode.java @@ -22,5 +22,6 @@ */ public enum PersistentStoreMode { PERSISTENT, - BLOB_PERSISTENT + BLOB_PERSISTENT, + PERSISTENT_TRANSACTION } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/store/ZookeeperPersistentStore.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/store/ZookeeperPersistentStore.java index c3c993c7c85..1e7a73e69a8 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/store/ZookeeperPersistentStore.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/store/ZookeeperPersistentStore.java @@ -17,15 +17,6 @@ */ package org.apache.drill.exec.store.sys.store; -import java.io.IOException; -import java.util.Iterator; -import java.util.Map; - -import javax.annotation.Nullable; - -import org.apache.drill.shaded.guava.com.google.common.base.Function; -import org.apache.drill.shaded.guava.com.google.common.base.Preconditions; -import org.apache.drill.shaded.guava.com.google.common.collect.Iterators; import org.apache.curator.framework.CuratorFramework; import org.apache.drill.common.collections.ImmutableEntry; import org.apache.drill.common.exceptions.DrillRuntimeException; @@ -37,16 +28,24 @@ import org.apache.drill.exec.store.sys.PersistentStoreConfig; import org.apache.drill.exec.store.sys.PersistentStoreMode; import org.apache.drill.exec.store.sys.VersionedPersistentStore; +import org.apache.drill.shaded.guava.com.google.common.base.Function; +import org.apache.drill.shaded.guava.com.google.common.base.Preconditions; +import org.apache.drill.shaded.guava.com.google.common.collect.Iterators; import org.apache.zookeeper.CreateMode; +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.Iterator; +import java.util.Map; + /** * Zookeeper based implementation of {@link org.apache.drill.exec.store.sys.PersistentStore}. */ public class ZookeeperPersistentStore extends BasePersistentStore implements VersionedPersistentStore { private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ZookeeperPersistentStore.class); - private final PersistentStoreConfig config; - private final ZookeeperClient client; + protected final PersistentStoreConfig config; + protected final ZookeeperClient client; public ZookeeperPersistentStore(final CuratorFramework framework, final PersistentStoreConfig config) throws StoreException { this.config = Preconditions.checkNotNull(config); diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/store/ZookeeperTransactionalPersistenceStore.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/store/ZookeeperTransactionalPersistenceStore.java new file mode 100644 index 00000000000..a4ac4740f52 --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/store/ZookeeperTransactionalPersistenceStore.java @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.drill.exec.store.sys.store; + +import org.apache.curator.framework.CuratorFramework; +import org.apache.drill.exec.exception.StoreException; +import org.apache.drill.exec.serialization.InstanceSerializer; +import org.apache.drill.exec.store.sys.PersistentStoreConfig; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Implementation of {@link org.apache.drill.exec.store.sys.store.ZookeeperPersistentStore} to support transactional + * operation for put, delete, create and get. For other single blob operation it just uses the implementation from + * parent. + * @param + */ +public class ZookeeperTransactionalPersistenceStore extends ZookeeperPersistentStore { + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ZookeeperTransactionalPersistenceStore.class); + + public ZookeeperTransactionalPersistenceStore(final CuratorFramework framework, + final PersistentStoreConfig config) throws StoreException { + super(framework, config); + } + + public boolean putAsTransaction(Map blobsToPut, DataChangeVersion version) { + Map.Entry currentEntry = null; + boolean isSuccess = false; + try { + final InstanceSerializer serializer = config.getSerializer(); + Map serializedBlobsToPut = new HashMap<>(); + for (Map.Entry entry : blobsToPut.entrySet()) { + currentEntry = entry; + serializedBlobsToPut.put(currentEntry.getKey(), serializer.serialize(currentEntry.getValue())); + } + client.putAsTransaction(serializedBlobsToPut, version); + isSuccess = true; + } catch (IOException ex) { + logger.error("Failed to serialize the blobs passed to write in a single transaction", ex); + } catch (Exception ex) { + logger.error("Failed to put the blobs in a single transaction", ex); + } + return isSuccess; + } + + public boolean putAsTransaction(Map blobsToPut) { + return putAsTransaction(blobsToPut, null); + } + + public boolean deleteAsTransaction(List blobPathsToDelete) { + boolean isSuccess = false; + try { + client.deleteAsTransaction(blobPathsToDelete); + isSuccess = true; + } catch (Exception ex) { + logger.error("Failed to delete on or more blobs in a single transaction", ex); + } + return isSuccess; + } + + public Map getAllOrNone(List blobsToGet, DataChangeVersion version) { + final Map blobsRequested = new HashMap<>(); + String currentBlob = ""; + try { + for (String blobName : blobsToGet) { + currentBlob = blobName; + final V blobData = get(currentBlob, true, version); + blobsRequested.put(currentBlob, blobData); + } + return blobsRequested; + } catch (Exception ex) { + // if here means there was error in getting one or more blob so consume the exception and just return null + logger.error("Error while getting one or more blob in getAllOrNone call. [Details: Requested blobs: {}, Issue " + + "Blob: {}]", String.join(",", blobsToGet), currentBlob); + } + return null; + } + + public Map getAllOrNone(List blobsToGet) { + return getAllOrNone(blobsToGet, null); + } + + public boolean createAsTransaction(List blobsToCreate) { + boolean isSuccess = false; + try { + client.createAsTransaction(blobsToCreate); + isSuccess = true; + } catch (Exception ex) { + logger.error("Failed to create one or more blobs in a single transaction", ex); + } + return isSuccess; + } +} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/store/provider/ZookeeperPersistentStoreProvider.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/store/provider/ZookeeperPersistentStoreProvider.java index de7221e8681..42ff0504a40 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/store/provider/ZookeeperPersistentStoreProvider.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/store/provider/ZookeeperPersistentStoreProvider.java @@ -17,23 +17,24 @@ */ package org.apache.drill.exec.store.sys.store.provider; -import java.io.IOException; - -import org.apache.drill.shaded.guava.com.google.common.annotations.VisibleForTesting; import org.apache.curator.framework.CuratorFramework; import org.apache.drill.common.config.DrillConfig; import org.apache.drill.exec.coord.zk.ZKClusterCoordinator; import org.apache.drill.exec.exception.StoreException; import org.apache.drill.exec.store.dfs.DrillFileSystem; import org.apache.drill.exec.store.sys.PersistentStore; -import org.apache.drill.exec.store.sys.PersistentStoreRegistry; import org.apache.drill.exec.store.sys.PersistentStoreConfig; +import org.apache.drill.exec.store.sys.PersistentStoreRegistry; import org.apache.drill.exec.store.sys.VersionedPersistentStore; import org.apache.drill.exec.store.sys.store.LocalPersistentStore; import org.apache.drill.exec.store.sys.store.VersionedDelegatingStore; import org.apache.drill.exec.store.sys.store.ZookeeperPersistentStore; +import org.apache.drill.exec.store.sys.store.ZookeeperTransactionalPersistenceStore; +import org.apache.drill.shaded.guava.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.fs.Path; +import java.io.IOException; + public class ZookeeperPersistentStoreProvider extends BasePersistentStoreProvider { // private static final Logger logger = LoggerFactory.getLogger(ZookeeperPersistentStoreProvider.class); @@ -69,14 +70,23 @@ public PersistentStore getOrCreateStore(final PersistentStoreConfig co switch(config.getMode()){ case BLOB_PERSISTENT: return new LocalPersistentStore<>(fs, blobRoot, config); - case PERSISTENT: - final ZookeeperPersistentStore store = new ZookeeperPersistentStore<>(curator, config); - try { - store.start(); - } catch (Exception e) { - throw new StoreException("unable to start zookeeper store", e); - } - return store; + case PERSISTENT: + final ZookeeperPersistentStore store = new ZookeeperPersistentStore<>(curator, config); + try { + store.start(); + } catch (Exception e) { + throw new StoreException("unable to start zookeeper store", e); + } + return store; + case PERSISTENT_TRANSACTION: + final ZookeeperTransactionalPersistenceStore transactionStore = + new ZookeeperTransactionalPersistenceStore<>(curator, config); + try { + transactionStore.start(); + } catch (Exception e) { + throw new StoreException("unable to start zookeeper transactional store", e); + } + return transactionStore; default: throw new IllegalStateException(); } @@ -95,6 +105,15 @@ public VersionedPersistentStore getOrCreateVersionedStore(final Persisten throw new StoreException("unable to start zookeeper store", e); } return store; + case PERSISTENT_TRANSACTION: + final ZookeeperTransactionalPersistenceStore transactionStore = + new ZookeeperTransactionalPersistenceStore<>(curator, config); + try { + transactionStore.start(); + } catch (Exception e) { + throw new StoreException("unable to start versioned zookeeper transactional store", e); + } + return transactionStore; default: throw new IllegalStateException(); } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java index 7915843ebb4..3447c4f2d45 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java @@ -75,7 +75,14 @@ public class WorkManager implements AutoCloseable { */ private final ConcurrentMap runningFragments = Maps.newConcurrentMap(); - private final ConcurrentMap queries = Maps.newConcurrentMap(); + // Map of all the queries which are admitted by queue leader + private final ConcurrentMap runningQueries = Maps.newConcurrentMap(); + // Map of all the queries which are waiting for admit response from the queue leader + private final ConcurrentMap waitingQueries = Maps.newConcurrentMap(); + // Lock to protect access of both waiting queries and running queries. This is needed to avoid race conditions For + // Example: when query cancellation is received and at same time query is moved between 2 maps + private Lock queryMapAccess = new ReentrantLock(); + private final BootStrapContext bContext; private DrillbitContext dContext; @@ -193,10 +200,15 @@ public void waitToExit(final boolean forcefulShutdown) { logger.warn("Timed out after {} millis. Shutting down before all fragments and foremen " + "have completed.", EXIT_TIMEOUT_MS); - for (QueryId queryId: queries.keySet()) { + for (QueryId queryId: runningQueries.keySet()) { logger.warn("Query {} is still running.", QueryIdHelper.getQueryId(queryId)); } + for (QueryId queryId: waitingQueries.keySet()) { + logger.warn("Query {} is in waiting queue and has not yet started running.", + QueryIdHelper.getQueryId(queryId)); + } + for (FragmentHandle fragmentHandle: runningFragments.keySet()) { logger.warn("Fragment {} is still running.", QueryIdHelper.getQueryIdentifier(fragmentHandle)); } @@ -212,7 +224,12 @@ public void waitToExit(final boolean forcefulShutdown) { } private boolean areQueriesAndFragmentsEmpty() { - return queries.isEmpty() && runningFragments.isEmpty(); + try { + queryMapAccess.lock(); + return runningQueries.isEmpty() && waitingQueries.isEmpty() && runningFragments.isEmpty(); + } finally { + queryMapAccess.unlock(); + } } /** @@ -222,7 +239,8 @@ private void indicateIfSafeToExit() { isEmptyLock.lock(); try { if (isEmptyCondition != null) { - logger.info("Waiting for {} running queries before shutting down.", queries.size()); + logger.info("Waiting for {} running queries before shutting down.", runningQueries.size()); + logger.info("Waiting for {} waiting queries before shutting down.", waitingQueries.size()); logger.info("Waiting for {} running fragments before shutting down.", runningFragments.size()); if (areQueriesAndFragmentsEmpty()) { @@ -239,10 +257,16 @@ private void indicateIfSafeToExit() { * shutdown request is triggered. */ public synchronized Map getRemainingQueries() { - Map queriesInfo = new HashMap(); - queriesInfo.put("queriesCount", queries.size()); - queriesInfo.put("fragmentsCount", runningFragments.size()); - return queriesInfo; + try { + queryMapAccess.lock(); + final Map queriesInfo = new HashMap(); + queriesInfo.put("queriesCount", runningQueries.size() + waitingQueries.size()); + queriesInfo.put("fragmentsCount", runningFragments.size()); + return queriesInfo; + } finally { + queryMapAccess.unlock(); + } + } /** @@ -250,12 +274,32 @@ public synchronized Map getRemainingQueries() { */ public class WorkerBee { public void addNewForeman(final Foreman foreman) { - queries.put(foreman.getQueryId(), foreman); + try { + queryMapAccess.lock(); + // Always put foreman inside waiting queries map first. Only after query is admitted by leader or enqueue + // returns true in case when RM is disabled the query will be moved to running map + waitingQueries.put(foreman.getQueryId(), foreman); + } finally { + queryMapAccess.unlock(); + } // We're relying on the Foreman to clean itself up with retireForeman(). executor.execute(foreman); } + public void moveToRunningQueries(QueryId queryId) { + try { + queryMapAccess.lock(); + final Foreman foremanForQuery = waitingQueries.get(queryId); + waitingQueries.remove(queryId); + if (foremanForQuery != null) { + runningQueries.put(queryId, foremanForQuery); + } + } finally { + queryMapAccess.unlock(); + } + } + /** * Add a self contained runnable work to executor service. * @param runnable @@ -267,8 +311,9 @@ public void addNewWork(final Runnable runnable) { public boolean cancelForeman(final QueryId queryId, DrillUserPrincipal principal) { Preconditions.checkNotNull(queryId); - final Foreman foreman = queries.get(queryId); + final Foreman foreman = getForemanForQueryId(queryId); if (foreman == null) { + // Foreman not found in both running and waiting queries map return false; } @@ -316,7 +361,13 @@ public void retireForeman(final Foreman foreman) { Preconditions.checkNotNull(foreman); final QueryId queryId = foreman.getQueryId(); - final boolean wasRemoved = queries.remove(queryId, foreman); + final boolean wasRemoved; + try { + queryMapAccess.lock(); + wasRemoved = runningQueries.remove(queryId, foreman) || waitingQueries.remove(queryId, foreman); + } finally { + queryMapAccess.unlock(); + } if (!wasRemoved) { logger.warn("Couldn't find retiring Foreman for query " + queryId); @@ -326,7 +377,14 @@ public void retireForeman(final Foreman foreman) { } public Foreman getForemanForQueryId(final QueryId queryId) { - return queries.get(queryId); + final Foreman foreman; + try { + queryMapAccess.lock(); + foreman = runningQueries.getOrDefault(queryId, waitingQueries.getOrDefault(queryId, null)); + } finally { + queryMapAccess.unlock(); + } + return foreman; } public DrillbitContext getContext() { @@ -391,7 +449,16 @@ public void receiveRuntimeFilter(final RuntimeFilterWritable runtimeFilter) { runtimeFilter.retainBuffers(1); //to foreman if (toForeman) { - Foreman foreman = queries.get(queryId); + // Foreman should be in runningQueries map only not in waitingQueries map since RTF will be received if query + // is actually executing + final Foreman foreman; + try { + queryMapAccess.lock(); + foreman = runningQueries.get(queryId); + } finally { + queryMapAccess.unlock(); + } + if (foreman != null) { executor.execute(new Runnable() { @Override @@ -426,9 +493,9 @@ public void run() { } /** - * Periodically gather current statistics. {@link QueryManager} uses a FragmentStatusListener to - * maintain changes to state, and should be current. However, we want to collect current statistics - * about RUNNING queries, such as current memory consumption, number of rows processed, and so on. + * Periodically gather current statistics. {@link org.apache.drill.exec.work.foreman.QueryManager} uses a + * FragmentStatusListener to maintain changes to state, and should be current. However, we want to collect current + * statistics about RUNNING queries, such as current memory consumption, number of rows processed, and so on. * The FragmentStatusListener only tracks changes to state, so the statistics kept there will be * stale; this thread probes for current values. * 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 df690444b3a..479df1f8385 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 @@ -17,9 +17,6 @@ */ package org.apache.drill.exec.work.foreman; -import org.apache.drill.exec.work.filter.RuntimeFilterRouter; -import org.apache.drill.shaded.guava.com.google.common.base.Preconditions; -import org.apache.drill.shaded.guava.com.google.common.collect.Lists; import com.google.protobuf.InvalidProtocolBufferException; import io.netty.channel.ChannelFuture; import io.netty.util.concurrent.Future; @@ -61,9 +58,12 @@ import org.apache.drill.exec.util.Pointer; import org.apache.drill.exec.work.QueryWorkUnit; import org.apache.drill.exec.work.WorkManager.WorkerBee; +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; +import org.apache.drill.shaded.guava.com.google.common.base.Preconditions; +import org.apache.drill.shaded.guava.com.google.common.collect.Lists; import org.codehaus.jackson.map.ObjectMapper; import java.io.IOException; @@ -254,35 +254,47 @@ public void run() { } queryText = queryRequest.getPlan(); - queryStateProcessor.moveToState(QueryState.PLANNING, null); try { injector.injectChecked(queryContext.getExecutionControls(), "run-try-beginning", ForemanException.class); - - // convert a run query request into action - switch (queryRequest.getType()) { - case LOGICAL: - parseAndRunLogicalPlan(queryRequest.getPlan()); - break; - case PHYSICAL: - parseAndRunPhysicalPlan(queryRequest.getPlan()); - break; - case SQL: - final String sql = queryRequest.getPlan(); - // log query id, username and query text before starting any real work. Also, put - // them together such that it is easy to search based on query id - logger.info("Query text for query with id {} issued by {}: {}", queryIdString, - queryContext.getQueryUserName(), sql); - runSQL(sql); - break; - case EXECUTION: - runFragment(queryRequest.getFragmentsList()); - break; - case PREPARED_STATEMENT: - runPreparedStatement(queryRequest.getPreparedStatementHandle()); - break; - default: - throw new IllegalStateException(); + switch (getState()) { + case PREPARING: + queryStateProcessor.moveToState(QueryState.PLANNING, null); + // convert a run query request into action + switch (queryRequest.getType()) { + case LOGICAL: + parseAndRunLogicalPlan(queryRequest.getPlan()); + break; + case PHYSICAL: + parseAndRunPhysicalPlan(queryRequest.getPlan()); + break; + case SQL: + final String sql = queryRequest.getPlan(); + // log query id, username and query text before starting any real work. Also, put + // them together such that it is easy to search based on query id + logger.info("Query text for query with id {} issued by {}: {}", queryIdString, + queryContext.getQueryUserName(), sql); + runSQL(sql); + break; + case EXECUTION: + runFragment(queryRequest.getFragmentsList()); + break; + case PREPARED_STATEMENT: + runPreparedStatement(queryRequest.getPreparedStatementHandle()); + break; + default: + throw new IllegalStateException(); + } + break; + case ENQUEUED: + startAdmittedQuery(); + break; + case STARTING: + reserveAndRunFragments(); + break; + default: + throw new IllegalStateException(String.format("Foreman object is not expected to be in this state %s inside " + + "run method", getState())); } injector.injectChecked(queryContext.getExecutionControls(), "run-try-end", ForemanException.class); } catch (final ForemanException e) { @@ -302,7 +314,7 @@ public void run() { queryStateProcessor.moveToState(QueryState.FAILED, e); } catch (AssertionError | Exception ex) { queryStateProcessor.moveToState(QueryState.FAILED, - new ForemanException("Unexpected exception during fragment initialization: " + ex.getMessage(), ex)); + new ForemanException("Unexpected exception during fragment initialization: " + ex.getMessage(), ex)); } finally { // restore the thread's original name currentThread.setName(originalName); @@ -310,8 +322,8 @@ public void run() { /* * Note that despite the run() completing, the Foreman continues to exist, and receives - * events (indirectly, through the QueryManager's use of stateListener), about fragment - * completions. It won't go away until everything is completed, failed, or cancelled. + * events about its enqueued successfully or not, (indirectly, through the QueryManager's use of stateListener), + * about fragment completions. It won't go away until everything is completed, failed, or cancelled. */ } @@ -478,9 +490,36 @@ private void runFragment(List fragmentsList) throws ExecutionSetup * Moves query to RUNNING state. */ private void startQueryProcessing() { - enqueue(); - runFragments(); - queryStateProcessor.moveToState(QueryState.RUNNING, null); + if (!enqueue()) { + // Since enqueue is async call based on response from scheduler the query will be in wait, execute or fail state + return; + } + startAdmittedQuery(); + } + + private void startAdmittedQuery() { + // move query into the running map + fragmentsRunner.getBee().moveToRunningQueries(queryId); + queryRM.updateState(QueryResourceManager.QueryRMState.ADMITTED); + queryStateProcessor.moveToState(QueryState.STARTING, null); + reserveAndRunFragments(); + } + + private void reserveAndRunFragments() { + // Now try to reserve the resources required by this query + try { + // TODO: pass parameters for reserveResources + if (!queryRM.reserveResources(null, queryId)) { + // 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); + return; + } + runFragments(); + } catch (Exception ex) { + queryStateProcessor.moveToState(QueryState.FAILED, ex); + } } /** @@ -488,14 +527,16 @@ private void startQueryProcessing() { * Foreman run will be blocked until query is enqueued. * In case of failures (ex: queue timeout exception) will move query to FAILED state. */ - private void enqueue() { + private boolean enqueue() { queryStateProcessor.moveToState(QueryState.ENQUEUED, null); - try { - queryRM.admit(); - queryStateProcessor.moveToState(QueryState.STARTING, null); + if (queryRM.admit() == QueryResourceManager.QueryAdmitResponse.WAIT_FOR_RESPONSE) { + return false; + } + return true; } catch (QueueTimeoutException | QueryQueueException e) { queryStateProcessor.moveToState(QueryState.FAILED, e); + return false; } finally { String queueName = queryRM.queueName(); queryManager.setQueueName(queueName == null ? "Unknown" : queueName); @@ -527,6 +568,7 @@ private void runFragments() { */ startProcessingEvents(); } + queryStateProcessor.moveToState(QueryState.RUNNING, null); } /** 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 5c0f71d8d8c..892fa7d8b27 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 @@ -21,8 +21,14 @@ 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.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.Foreman; +import java.util.Map; + /** * Represents a default resource manager for clusters that do not provide query * queues. Without queues to provide a hard limit on the query admission rate, @@ -33,7 +39,6 @@ public class DefaultResourceManager implements ResourceManager { public static class DefaultQueryResourceManager implements QueryResourceManager { - private final DefaultResourceManager rm; private final QueryContext queryContext; @@ -47,14 +52,37 @@ public void setCost(double cost) { // Nothing to do by default. } + @Override + public void setCost(Map costOnAssignedEndpoints) { + // Nothing to do by default + } + @Override public QueryParallelizer getParallelizer(boolean memoryPlanning){ return new DefaultParallelizer(memoryPlanning, queryContext); } - @Override - public void admit() { + public QueryAdmitResponse admit() { // No queueing by default + return QueryAdmitResponse.ADMITTED; + } + + public boolean reserveResources(QueryQueueConfig selectedQueue, UserBitShared.QueryId queryId) throws Exception { + return true; + } + + @Override + public QueryQueueConfig selectQueue(NodeResources maxNodeResource) throws QueueSelectionException { + throw new UnsupportedOperationException("Queue is not supported in default resource manager"); + } + + @Override + public String getLeaderId() { + throw new UnsupportedOperationException("Leader is not supported in the DefaultResourceManager"); + } + + public void updateState(QueryRMState newState) { + // no op since Default QueryRM doesn't have any state machine } @Override @@ -102,6 +130,11 @@ 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 6932bcdb20e..48b9d81487a 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 @@ -20,14 +20,39 @@ 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.exception.StoreException; import org.apache.drill.exec.ops.QueryContext; -import org.apache.drill.exec.planner.fragment.QueryParallelizer; 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.resourcemgr.NodeResources; +import org.apache.drill.exec.resourcemgr.config.QueryQueueConfig; +import org.apache.drill.exec.resourcemgr.config.RMCommonDefaults; import org.apache.drill.exec.resourcemgr.config.ResourcePoolTree; import org.apache.drill.exec.resourcemgr.config.ResourcePoolTreeImpl; +import org.apache.drill.exec.resourcemgr.config.exception.QueueSelectionException; import org.apache.drill.exec.resourcemgr.config.exception.RMConfigException; +import org.apache.drill.exec.resourcemgr.exception.QueueWaitTimeoutExpired; +import org.apache.drill.exec.resourcemgr.rmblobmgr.RMBlobStoreManager; +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.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.util.Collection; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Map; +import java.util.PriorityQueue; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +import static org.apache.drill.exec.ExecConstants.RM_WAIT_THREAD_INTERVAL; + public class DistributedResourceManager implements ResourceManager { private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DistributedResourceManager.class); @@ -37,9 +62,24 @@ public class DistributedResourceManager implements ResourceManager { private final DrillbitContext context; private final DrillConfig rmConfig; + public final long memoryPerNode; + public final int cpusPerNode; + private final WaitQueueThread waitQueueThread; + + 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<>(); + + // 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()); + public DistributedResourceManager(DrillbitContext context) throws DrillRuntimeException { memoryPerNode = DrillConfig.getMaxDirectMemory(); cpusPerNode = Runtime.getRuntime().availableProcessors(); @@ -49,11 +89,41 @@ public DistributedResourceManager(DrillbitContext context) throws DrillRuntimeEx rmPoolTree = new ResourcePoolTreeImpl(rmConfig, DrillConfig.getMaxDirectMemory(), Runtime.getRuntime().availableProcessors(), 1); logger.debug("Successfully parsed RM config \n{}", rmConfig.getConfig(ResourcePoolTreeImpl.ROOT_POOL_CONFIG_KEY)); + Set leafQueues = rmPoolTree.getAllLeafQueues().keySet(); + for (String leafQueue : leafQueues) { + waitingQueuesForAdmittedQuery.put(leafQueue, new PriorityQueue<>(waitTimeComparator)); + } + this.rmBlobStoreManager = new RMConsistentBlobStoreManager(context, rmPoolTree.getAllLeafQueues().values()); + + // start the wait thread + 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); } 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); + } catch (StoreException ex) { + throw new DrillRuntimeException("Failed while creating the blob store manager for managing RM state blobs", ex); + } + } + + 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; @@ -66,49 +136,72 @@ public int cpusPerNode() { @Override public QueryResourceManager newQueryRM(Foreman foreman) { - return new QueuedQueryResourceManager(this, foreman); + return new DistributedQueryRM(this, foreman); + } + + @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); + } + + private void reserveResources(Map queryResourceAssignment, + QueryQueueConfig selectedQueue, String leaderId, String queryId, + String foremanNode) throws Exception { + //rmBlobStoreManager.reserveResources(); + } + + private void freeResources(Map queryResourceAssignment, QueryQueueConfig selectedQueue, + String leaderId, String queryId, String foremanNode) throws Exception { + } public ResourcePoolTree getRmPoolTree() { return rmPoolTree; } + @Override + public void close() { + } + + public class DistributedQueryRM implements QueryResourceManager { - /** - * 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 - * with a Foreman: a single thread at plan time, and a single event (in some - * thread) at query completion time. Because of these semantics, no - * synchronization is needed within this class. - */ + private final DistributedResourceManager drillRM; - public static class QueuedQueryResourceManager implements QueryResourceManager { + private final QueryContext context; private final Foreman foreman; - private final QueryContext queryContext; - private double queryCost; - private final DistributedResourceManager rm; - public QueuedQueryResourceManager(final DistributedResourceManager rm, - final Foreman foreman) { - this.foreman = foreman; - this.queryContext = foreman.getQueryContext(); - this.rm = rm; + private QueryRMState currentState; + + private Stopwatch waitStartTime; + + private Map assignedEndpointsCost; + + DistributedQueryRM(ResourceManager resourceManager, Foreman queryForeman) { + this.drillRM = (DistributedResourceManager) resourceManager; + this.context = queryForeman.getQueryContext(); + this.foreman = queryForeman; + currentState = QueryRMState.STARTED; } @Override - public void setCost(double cost) { - this.queryCost = cost; + public boolean hasQueue() { + return true; } @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.queryContext); + public void setCost(double cost) { + throw new UnsupportedOperationException("DistributedQueryRM doesn't support cost in double format"); } - @Override - public void admit() throws QueryQueue.QueueTimeoutException, QueryQueue.QueryQueueException { + public void setCost(Map costOnAssignedEndpoints) { + // Should be called when queryRM is in STARTED state + Preconditions.checkState(currentState == QueryRMState.STARTED, + "Cost is being set when queryRM is in %s state", currentState.toString()); + assignedEndpointsCost = costOnAssignedEndpoints; } public long queryMemoryPerNode() { @@ -121,19 +214,119 @@ public long minimumOperatorMemory() { } @Override - public void exit() { + 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); } @Override - public boolean hasQueue() { return true; } + public QueryAdmitResponse admit() throws QueueTimeoutException, QueryQueueException { + // TODO: for now it will just return since leader election is not available + // Once leader election support is there we will throw exception in case of error + // otherwise just return + updateState(QueryRMState.ENQUEUED); + return QueryAdmitResponse.ADMITTED; + } @Override public String queueName() { return ""; } + + @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 + } + + @Override + public String getLeaderId() { + // TODO: Return emoty string for now + return ""; + } + + public boolean reserveResources(QueryQueueConfig selectedQueue, UserBitShared.QueryId queryId) throws Exception { + try { + 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); + 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 + if (waitStartTime == null) { + waitStartTime = Stopwatch.createStarted(); + } + // Check if wait time has expired before adding in waiting queue + final long timeElapsedWaiting = elapsedWaitTime(); + if (timeElapsedWaiting >= selectedQueue.getWaitTimeoutInMs()) { + // 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); + } + drillRM.addToWaitingQueue(this); + return false; + } catch (Exception ex) { + logger.error("Failed while reserving resources for this query", ex); + throw ex; + } + } + + private long elapsedWaitTime() { + return waitStartTime.elapsed(TimeUnit.MILLISECONDS); + } + + 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; + } + + @Override + public void exit() { + // 1. if queryRM is in admitted state: That means exit is called either when query is failed. When query is + // cancelled then exit will never be called in ADMITTED state of queryRM. Since then query will be in STARTING + // state and cancellation_requested event will be queued until query moves to running state. + // + // Because of above even though queryRM can be in waiting queue of Resource Pool, it doesn't have any race + // condition with cancel request. Since cancel will not be processed until queryRM moves to RESERVED_RESOURCES + // STATE as part of query moving to Running state + // + // In the failure case queryRM just needs to send message back to leader to release it's reserved slot. Message + // will be sent back to leader who admitted the query. So we don't have to read the blob as resources + // are not reserved yet and running count is not incremented. Also as part of failure since exit will be called + // on queryRM from failure handling thread, then waiting thread should always check the queryState before + // trying to reserve resource for it. If query is in terminal state then it should just log it and remove + // queryRM for that query from the waiting queue. + // + // 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. + } } - @Override - public void close() { + public static class WaitQueueThread extends Thread { + private final int refreshInterval; + + public WaitQueueThread(int waitInterval) { + setName("DistributedResourceManager.WaitThread"); + refreshInterval = waitInterval; + } + + @Override + public void run() { + while (true) { + try { + Thread.sleep(refreshInterval); + } catch (InterruptedException ex) { + logger.error("Thread {} is interrupted", getName()); + Thread.currentThread().interrupt(); + break; + } + } + } } } 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 391afa8c5cc..872ef50a244 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,6 +66,11 @@ 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 a741c97ce49..ef61bf39842 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,15 +18,47 @@ 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; import org.apache.drill.exec.work.foreman.rm.QueryQueue.QueryQueueException; import org.apache.drill.exec.work.foreman.rm.QueryQueue.QueueTimeoutException; +import java.util.Map; + /** * Extends a {@link QueryResourceAllocator} to provide queueing support. */ public interface QueryResourceManager { + enum QueryAdmitResponse { + UNKNOWN, + ADMITTED, + WAIT_FOR_RESPONSE; + + @Override + public String toString() { + return name().toLowerCase(); + } + } + + enum QueryRMState { + STARTED, + ENQUEUED, + ADMITTED, + RESERVED_RESOURCES, + RELEASED_RESOURCES, + DEQUEUED, + COMPLETED; + + @Override + public String toString() { + return name().toLowerCase(); + } + } + /** * Hint that this resource manager queues. Allows the Foreman * to short-circuit expensive logic if no queuing will actually @@ -44,6 +76,7 @@ public interface QueryResourceManager { void setCost(double cost); + 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 @@ -59,11 +92,9 @@ public interface QueryResourceManager { * approach.) * @throws QueryQueueException if something goes wrong with the * queue mechanism - * @throws QueueTimeoutException if the query timed out waiting to - * be admitted. */ - void admit() throws QueueTimeoutException, QueryQueueException; + QueryAdmitResponse admit() throws QueueTimeoutException, QueryQueueException; /** @@ -80,6 +111,20 @@ public interface QueryResourceManager { long minimumOperatorMemory(); + /** + * Updates the state machine of queryRM + * @param newState new target state + */ + void updateState(QueryRMState newState); + + /** + * Called to reserve resources required by query. Updates the queryRM state to RESERVED_RESOURCES if successful + */ + boolean reserveResources(QueryQueueConfig selectedQueue, UserBitShared.QueryId queryId) throws Exception; + + QueryQueueConfig selectQueue(NodeResources maxNodeResource) throws QueueSelectionException; + + String getLeaderId(); /** * Mark the query as completing, giving up its slot in the * cluster. Releases any lease that may be held for a system with queues. 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 821bbf2f2fb..ac9554971dc 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,5 +51,7 @@ 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/ThrottledResourceManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/ThrottledResourceManager.java index 3572dd7bdb1..9f7ff6fb77f 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 @@ -18,14 +18,20 @@ package org.apache.drill.exec.work.foreman.rm; import org.apache.drill.exec.ops.QueryContext; -import org.apache.drill.exec.planner.fragment.ZKQueueParallelizer; 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; import org.apache.drill.exec.server.DrillbitContext; 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.QueueLease; import org.apache.drill.exec.work.foreman.rm.QueryQueue.QueueTimeoutException; +import java.util.Map; + /** * Global resource manager that provides basic admission control (AC) via a * configured queue: either the Zookeeper-based distributed queue or the @@ -75,15 +81,21 @@ public void setCost(double cost) { this.queryCost = cost; } + @Override + public void setCost(Map costOnAssignedEndpoints) { + throw new UnsupportedOperationException("QueuedQueryResourceManager doesn't use this implementation for setting" + + " cost"); + } + @Override public QueryParallelizer getParallelizer(boolean planHasMemory) { // currently memory planning is disabled. Enable it once the RM functionality is fully implemented. return new ZKQueueParallelizer(planHasMemory, this, this.queryContext); } - @Override - public void admit() throws QueueTimeoutException, QueryQueueException { + public QueryAdmitResponse admit() throws QueueTimeoutException, QueryQueueException { lease = rm.queue().enqueue(foreman.getQueryId(), queryCost); + return QueryAdmitResponse.ADMITTED; } public long queryMemoryPerNode() { @@ -102,6 +114,26 @@ public long minimumOperatorMemory() { return rm.minimumOperatorMemory(); } + @Override + public boolean reserveResources(QueryQueueConfig selectedQueue, UserBitShared.QueryId queryId) throws Exception { + // no op + return true; + } + + @Override + public QueryQueueConfig selectQueue(NodeResources maxNodeResource) throws QueueSelectionException { + throw new UnsupportedOperationException("Select queue is not supported in QueuedQueryResourceManager"); + } + + @Override + public String getLeaderId() { + throw new UnsupportedOperationException("Leader is not supported in QueuedQueryResourceManager"); + } + + public void updateState(QueryRMState state) { + // no-op Doesn't support any state machine + } + @Override public void exit() { if (lease != null) { @@ -143,6 +175,12 @@ 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/resources/drill-module.conf b/exec/java-exec/src/main/resources/drill-module.conf index b2ff4a594cb..82c86c5f040 100644 --- a/exec/java-exec/src/main/resources/drill-module.conf +++ b/exec/java-exec/src/main/resources/drill-module.conf @@ -30,7 +30,8 @@ drill { org.apache.drill.exec.server.rest.auth.DrillHttpConstraintSecurityHandler, org.apache.drill.exec.store.dfs.FormatPlugin, org.apache.drill.exec.store.StoragePlugin, - org.apache.drill.exec.coord.zk.ZKACLProvider + org.apache.drill.exec.coord.zk.ZKACLProvider, + org.apache.drill.exec.resourcemgr.rmblobmgr.rmblob.RMStateBlob ], annotations: ${?drill.classpath.scanning.annotations} [ @@ -45,7 +46,8 @@ drill { org.apache.drill.exec.rpc.user.security, org.apache.drill.exec.rpc.security, org.apache.drill.exec.server.rest.auth, - org.apache.drill.exec.coord.zk + org.apache.drill.exec.coord.zk, + org.apache.drill.exec.resourcemgr.rmblobmgr.rmblob ], // caches scanned result during build time 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 new file mode 100644 index 00000000000..89657e13e70 --- /dev/null +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/resourcemgr/RMBlobManagerTest.java @@ -0,0 +1,171 @@ +/* + * 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 com.typesafe.config.Config; +import com.typesafe.config.ConfigFactory; +import com.typesafe.config.ConfigValueFactory; +import org.apache.drill.exec.ExecConstants; +import org.apache.drill.exec.resourcemgr.config.QueryQueueConfig; +import org.apache.drill.exec.resourcemgr.config.QueryQueueConfigImpl; +import org.apache.drill.exec.resourcemgr.rmblobmgr.RMConsistentBlobStoreManager; +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; +import org.apache.drill.exec.resourcemgr.rmblobmgr.rmblob.QueueLeadershipBlob; +import org.apache.drill.exec.resourcemgr.rmblobmgr.rmblob.RMStateBlob; +import org.apache.drill.exec.server.DrillbitContext; +import org.apache.drill.test.BaseDirTestWatcher; +import org.apache.drill.test.ClusterFixture; +import org.apache.drill.test.ClusterFixtureBuilder; +import org.apache.drill.test.DrillTest; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; + +import static org.junit.Assert.assertEquals; + +public class RMBlobManagerTest extends DrillTest { + //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RMBlobManagerTest.class); + @Rule + public final BaseDirTestWatcher dirTestWatcher = new BaseDirTestWatcher(); + + private Config queueConfig; + + private RMConsistentBlobStoreManager rmConsistentBlobStoreManager; + + private ClusterStateBlob clusterStateBlob; + + private QueueLeadershipBlob queueLeadershipBlob; + + private ForemanQueueUsageBlob foremanQueueUsageBlob; + + private final List leafQueues = new ArrayList<>(); + + @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); + } + + @Test + public void testRMStateBlobSerDe() throws Exception { + ClusterFixtureBuilder fixtureBuilder = ClusterFixture.builder(dirTestWatcher) + .configProperty(ExecConstants.DRILL_PORT_HUNT, true) + .withLocalZk(); + + 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 (QueryQueueConfig leafQueue : leafQueues) { + String blobName = ForemanQueueUsageBlob.NAME + "_" + leafQueue.getQueueName(); + blobsToSerialize.put(blobName, foremanQueueUsageBlob); + } + + final Map serializedBlobs = rmConsistentBlobStoreManager.serializePassedInBlob(blobsToSerialize); + final Map deserializedBlobs = rmConsistentBlobStoreManager.deserializeRMStateBlobs(serializedBlobs); + + for (Map.Entry blobEntry : deserializedBlobs.entrySet()) { + final RMStateBlob actualBlob = blobEntry.getValue(); + assertEquals(blobsToSerialize.get(blobEntry.getKey()), actualBlob); + } + } + } + + public void testBlobManagerReserveApi() throws Exception { + ClusterFixtureBuilder fixtureBuilder = ClusterFixture.builder(dirTestWatcher) + .configProperty(ExecConstants.DRILL_PORT_HUNT, true) + .withLocalZk(); + + try (ClusterFixture cluster = fixtureBuilder.build()) { + DrillbitContext context = cluster.drillbit().getContext(); + final RMConsistentBlobStoreManager rmManager = new RMConsistentBlobStoreManager(context, leafQueues); + + } + } +} From 2b6a91a212e592a66bdad175f7b3fb8041c2ae6b Mon Sep 17 00:00:00 2001 From: HanumathRao Date: Wed, 17 Apr 2019 14:26:02 -0700 Subject: [PATCH 4/7] DRILL-7193: Integration changes of the Distributed RM queue configuration with Simple Parallelizer. Integration changes with new DistributedRM queue configuration. a) Remove the redundant NodeResource and merge the additional member functions with the NodeResources class. b) Added new UUID logic and selection of a queue based on the memory requirement during parallelization phase. c) Changed proto definitions to set the UUID of a drillbit. d) Implementation of new DrillNode Wrapper over DrillbitEndpoint to fix the equality comparisions between DrillbitEndpoints. --- .../org/apache/drill/common/DrillNode.java | 88 +++++++ .../apache/drill/exec/ops/QueryContext.java | 19 +- .../drill/exec/planner/cost/NodeResource.java | 73 ------ .../planner/fragment/DefaultParallelizer.java | 4 +- .../DistributedQueueParallelizer.java | 64 ++++-- .../planner/fragment/MemoryCalculator.java | 6 +- .../planner/fragment/QueryParallelizer.java | 6 +- .../planner/fragment/SimpleParallelizer.java | 27 ++- .../drill/exec/planner/fragment/Wrapper.java | 14 +- .../planner/fragment/ZKQueueParallelizer.java | 2 +- .../contrib/SplittingParallelizer.java | 18 +- .../drill/exec/resourcemgr/NodeResources.java | 27 ++- .../drill/exec/server/DrillbitContext.java | 5 + .../drill/exec/work/foreman/Foreman.java | 3 +- .../rm/DistributedResourceManager.java | 2 +- .../drill/exec/work/user/PlanSplitter.java | 4 +- .../exec/physical/impl/TestLocalExchange.java | 2 +- .../partitionsender/TestPartitionSender.java | 2 +- .../exec/planner/rm/TestMemoryCalculator.java | 72 +++--- .../drill/exec/pop/TestFragmentChecker.java | 9 +- .../apache/drill/exec/proto/BitControl.java | 216 ++++++++++++++++-- .../drill/exec/proto/SchemaBitControl.java | 7 + .../drill/exec/proto/beans/PlanFragment.java | 22 ++ protocol/src/main/protobuf/BitControl.proto | 1 + 24 files changed, 494 insertions(+), 199 deletions(-) create mode 100644 exec/java-exec/src/main/java/org/apache/drill/common/DrillNode.java delete mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/NodeResource.java diff --git a/exec/java-exec/src/main/java/org/apache/drill/common/DrillNode.java b/exec/java-exec/src/main/java/org/apache/drill/common/DrillNode.java new file mode 100644 index 00000000000..8af70a3132c --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/common/DrillNode.java @@ -0,0 +1,88 @@ +/* + * 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.common; + +import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint; + +/** + * DrillNode encapsulates a drillendpoint. DrillbitEndpoint is a protobuf generated class which requires + * all the member variables to be equal for DrillbitEndpoints to be equal. DrillNode relaxes this requirement + * by only comparing required variables. + */ +public class DrillNode { + private final DrillbitEndpoint endpoint; + + public DrillNode(DrillbitEndpoint endpoint) { + this.endpoint = endpoint; + } + + public static DrillNode create(DrillbitEndpoint endpoint) { + return new DrillNode(endpoint); + } + + public boolean equals(Object other) { + if (!(other instanceof DrillNode)) { + return false; + } + + DrillbitEndpoint otherEndpoint = ((DrillNode) other).endpoint; + return endpoint.getAddress().equals(otherEndpoint.getAddress()) && + endpoint.getUserPort() == otherEndpoint.getUserPort() && + endpoint.getControlPort() == otherEndpoint.getControlPort() && + endpoint.getDataPort() == otherEndpoint.getDataPort() && + endpoint.getVersion().equals(otherEndpoint.getVersion()); + } + + @Override + public int hashCode() { + int hash = 41; + hash = (19 * hash) + endpoint.getDescriptor().hashCode(); + if (endpoint.hasAddress()) { + hash = (37 * hash) + endpoint.ADDRESS_FIELD_NUMBER; + hash = (53 * hash) + endpoint.getAddress().hashCode(); + } + if (endpoint.hasUserPort()) { + hash = (37 * hash) + endpoint.USER_PORT_FIELD_NUMBER; + hash = (53 * hash) + endpoint.getUserPort(); + } + if (endpoint.hasControlPort()) { + hash = (37 * hash) + endpoint.CONTROL_PORT_FIELD_NUMBER; + hash = (53 * hash) + endpoint.getControlPort(); + } + if (endpoint.hasDataPort()) { + hash = (37 * hash) + endpoint.DATA_PORT_FIELD_NUMBER; + hash = (53 * hash) + endpoint.getDataPort(); + } + if (endpoint.hasVersion()) { + hash = (37 * hash) + endpoint.VERSION_FIELD_NUMBER; + hash = (53 * hash) + endpoint.getVersion().hashCode(); + } + return hash; + } + + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + + return sb.append("endpoint address :") + .append(endpoint.getAddress()) + .append("endpoint user port: ") + .append(endpoint.getUserPort()).toString(); + } +} 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 97db9b241f8..ea783225016 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 @@ -17,12 +17,10 @@ */ package org.apache.drill.exec.ops; -import java.util.Collection; -import java.util.List; -import java.util.Map; - +import io.netty.buffer.DrillBuf; import org.apache.calcite.schema.SchemaPlus; import org.apache.drill.common.AutoCloseables; +import org.apache.drill.common.DrillNode; import org.apache.drill.common.config.DrillConfig; import org.apache.drill.common.config.LogicalPlanPersistence; import org.apache.drill.common.types.TypeProtos.MinorType; @@ -51,12 +49,13 @@ import org.apache.drill.exec.store.StoragePluginRegistry; import org.apache.drill.exec.testing.ExecutionControls; import org.apache.drill.exec.util.Utilities; - import org.apache.drill.shaded.guava.com.google.common.base.Function; import org.apache.drill.shaded.guava.com.google.common.collect.Lists; import org.apache.drill.shaded.guava.com.google.common.collect.Maps; -import io.netty.buffer.DrillBuf; +import java.util.Collection; +import java.util.List; +import java.util.Map; // TODO - consider re-name to PlanningContext, as the query execution context actually appears // in fragment contexts @@ -242,6 +241,14 @@ public Collection getOnlineEndpoints() { return drillbitContext.getBits(); } + /** + * TODO: Change it to use {@link DrillNode} instead of DrillbitEndpoint + * @return map of endpoint to UUIDs + */ + public Map getOnlineEndpointUUIDs() { + return drillbitContext.getOnlineEndpointUUIDs(); + } + public DrillConfig getConfig() { return drillbitContext.getConfig(); } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/NodeResource.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/NodeResource.java deleted file mode 100644 index ad7bc2e75f5..00000000000 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/NodeResource.java +++ /dev/null @@ -1,73 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.drill.exec.planner.cost; - -import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint; -import java.util.Map; - -/** - * This class abstracts the resources like cpu and memory used up by the operators. - * In future network resources can also be incorporated if required. - */ -public class NodeResource { - private long cpu; - private long memory; - - public NodeResource(long cpu, long memory) { - this.cpu = cpu; - this.memory = memory; - } - - public void add(NodeResource other) { - if (other == null) { - return; - } - this.cpu += other.cpu; - this.memory += other.memory; - } - - public long getMemory() { - return memory; - } - - // A utility function to merge the node resources from one drillbit map to other drillbit map. - public static Map merge(Map to, - Map from) { - to.entrySet().stream().forEach((toEntry) -> toEntry.getValue().add(from.get(toEntry.getKey()))); - return to; - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder(); - sb.append("CPU: ").append(cpu).append("Memory: ").append(memory); - return sb.toString(); - } - - public static NodeResource create() { - return create(0,0); - } - - public static NodeResource create(long cpu) { - return create(cpu,0); - } - - public static NodeResource create(long cpu, long memory) { - return new NodeResource(cpu, memory); - } -} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/DefaultParallelizer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/DefaultParallelizer.java index b298fa793a5..492d7f30a90 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/DefaultParallelizer.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/DefaultParallelizer.java @@ -22,8 +22,8 @@ import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint; import org.apache.drill.exec.util.memory.DefaultMemoryAllocationUtilities; -import java.util.Collection; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.function.BiFunction; @@ -51,7 +51,7 @@ public DefaultParallelizer(boolean memoryPlanning, long parallelizationThreshold @Override public void adjustMemory(PlanningSet planningSet, Set roots, - Collection activeEndpoints) { + Map onlineEndpointUUIDs) { if (planHasMemory) { return; } 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 009c750452c..c7c91f30b15 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 @@ -18,12 +18,16 @@ package org.apache.drill.exec.planner.fragment; import org.apache.commons.lang3.tuple.Pair; +import org.apache.drill.common.exceptions.ExecutionSetupException; import org.apache.drill.common.util.function.CheckedConsumer; 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.exec.planner.cost.NodeResource; 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; @@ -43,12 +47,14 @@ public class DistributedQueueParallelizer extends SimpleParallelizer { private final boolean planHasMemory; private final QueryContext queryContext; + private final QueryResourceManager rm; private final Map> operators; - public DistributedQueueParallelizer(boolean memoryPlanning, QueryContext queryContext) { + public DistributedQueueParallelizer(boolean memoryPlanning, QueryContext queryContext, QueryResourceManager queryRM) { super(queryContext); this.planHasMemory = memoryPlanning; this.queryContext = queryContext; + this.rm = queryRM; this.operators = new HashMap<>(); } @@ -75,38 +81,47 @@ public BiFunction getMemory() { * * @param planningSet context of the fragments. * @param roots root fragments. - * @param activeEndpoints currently active endpoints. + * @param onlineEndpointUUIDs currently active endpoints. * @throws PhysicalOperatorSetupException */ public void adjustMemory(PlanningSet planningSet, Set roots, - Collection activeEndpoints) throws PhysicalOperatorSetupException { + Map onlineEndpointUUIDs) throws ExecutionSetupException { if (planHasMemory) { return; } // total node resources for the query plan maintained per drillbit. - final Map totalNodeResources = - activeEndpoints.stream().collect(Collectors.toMap(x ->x, - x -> NodeResource.create())); + final Map totalNodeResources = + onlineEndpointUUIDs.keySet().stream().collect(Collectors.toMap(x ->x, + x -> NodeResources.create())); // list of the physical operators and their memory requirements per drillbit. final Map>> operators = - activeEndpoints.stream().collect(Collectors.toMap(x -> x, + onlineEndpointUUIDs.keySet().stream().collect(Collectors.toMap(x -> x, x -> new ArrayList<>())); for (Wrapper wrapper : roots) { traverse(wrapper, CheckedConsumer.throwingConsumerWrapper((Wrapper fragment) -> { MemoryCalculator calculator = new MemoryCalculator(planningSet, queryContext); fragment.getNode().getRoot().accept(calculator, fragment); - NodeResource.merge(totalNodeResources, fragment.getResourceMap()); + NodeResources.merge(totalNodeResources, fragment.getResourceMap()); operators.entrySet() .stream() .forEach((entry) -> entry.getValue() .addAll(calculator.getBufferedOperators(entry.getKey()))); })); } - //queryrm.selectQueue( pass the max node Resource) returns queue configuration. - Map>> memoryAdjustedOperators = ensureOperatorMemoryWithinLimits(operators, totalNodeResources, 10); + + QueryQueueConfig queueConfig = null; + 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) -> { Map memoryPerOperator = x.getValue().stream() .collect(Collectors.toMap(operatorLongPair -> operatorLongPair.getLeft(), @@ -114,6 +129,27 @@ public void adjustMemory(PlanningSet planningSet, Set roots, (mem_1, mem_2) -> (mem_1 + mem_2))); this.operators.put(x.getKey(), memoryPerOperator); }); + + this.rm.setCost(convertToUUID(totalNodeResources, onlineEndpointUUIDs)); + } + + private Map convertToUUID(Map nodeResourcesMap, + Map onlineEndpointUUIDs) { + Map nodeResourcesPerUUID = new HashMap<>(); + for (Map.Entry nodeResource : nodeResourcesMap.entrySet()) { + nodeResourcesPerUUID.put(onlineEndpointUUIDs.get(nodeResource.getKey()), nodeResource.getValue()); + } + return nodeResourcesPerUUID; + } + + private NodeResources max(Collection resources) { + NodeResources maxResource = null; + for (NodeResources resource : resources) { + if (maxResource == null || maxResource.getMemoryInBytes() < resource.getMemoryInBytes()) { + maxResource = resource; + } + } + return maxResource; } @@ -126,12 +162,12 @@ public void adjustMemory(PlanningSet planningSet, Set roots, */ private Map>> ensureOperatorMemoryWithinLimits(Map>> memoryPerOperator, - Map nodeResourceMap, int nodeLimit) { + Map nodeResourceMap, long nodeLimit) { // Get the physical operators which are above the node memory limit. Map>> onlyMemoryAboveLimitOperators = new HashMap<>(); memoryPerOperator.entrySet().stream().forEach((entry) -> { onlyMemoryAboveLimitOperators.putIfAbsent(entry.getKey(), new ArrayList<>()); - if (nodeResourceMap.get(entry.getKey()).getMemory() > nodeLimit) { + if (nodeResourceMap.get(entry.getKey()).getMemoryInBytes() > nodeLimit) { onlyMemoryAboveLimitOperators.get(entry.getKey()).addAll(entry.getValue()); } }); @@ -148,6 +184,8 @@ public void adjustMemory(PlanningSet planningSet, Set roots, return Pair.of(operatorMemory.getKey(), (long) Math.ceil(operatorMemory.getValue()/totalMemory * nodeLimit)); }).collect(Collectors.toList()); memoryAdjustedDrillbits.put(entry.getKey(), adjustedMemory); + NodeResources nodeResources = nodeResourceMap.get(entry.getKey()); + nodeResources.setMemoryInBytes(adjustedMemory.stream().mapToLong(Pair::getValue).sum()); } ); 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 443ab79cdae..94d319d4083 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,8 +23,8 @@ 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.exec.planner.cost.NodeResource; 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; @@ -70,11 +70,11 @@ private void merge(Wrapper currFrag, Map minorFragsPerDrillBit, Function, Long> getMemory) { - NodeResource.merge(currFrag.getResourceMap(), + NodeResources.merge(currFrag.getResourceMap(), minorFragsPerDrillBit.entrySet() .stream() .collect(Collectors.toMap((x) -> x.getKey(), - (x) -> NodeResource.create(0, + (x) -> NodeResources.create(0, getMemory.apply(x))))); } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/QueryParallelizer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/QueryParallelizer.java index 097f4b2493e..c6a581fbeb5 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/QueryParallelizer.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/QueryParallelizer.java @@ -25,7 +25,7 @@ import org.apache.drill.exec.server.options.OptionList; import org.apache.drill.exec.work.QueryWorkUnit; -import java.util.Collection; +import java.util.Map; /** * This class parallelizes the query plan. Once the optimizer finishes its job by producing a @@ -50,7 +50,7 @@ public interface QueryParallelizer extends ParallelizationParameters { * @param options List of all options that are set for the current session. * @param foremanNode Endpoint information of the foreman node. * @param queryId Unique ID of the query. - * @param activeEndpoints Currently active endpoints on which the plan can run. + * @param onlineEndpoints Currently active endpoints on which the plan can run. * @param rootFragment root of the fragment tree of the transformed physical plan * @param session user session object. * @param queryContextInfo query context. @@ -58,6 +58,6 @@ public interface QueryParallelizer extends ParallelizationParameters { * @throws ExecutionSetupException */ QueryWorkUnit generateWorkUnit(OptionList options, DrillbitEndpoint foremanNode, QueryId queryId, - Collection activeEndpoints, Fragment rootFragment, + Map onlineEndpoints, Fragment rootFragment, UserSession session, QueryContextInformation queryContextInfo) throws ExecutionSetupException; } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/SimpleParallelizer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/SimpleParallelizer.java index a434bf80b2a..aa4a4856e7b 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/SimpleParallelizer.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/SimpleParallelizer.java @@ -20,9 +20,11 @@ import java.util.ArrayList; import java.util.Collection; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.function.BiFunction; import java.util.function.Consumer; +import java.util.stream.Collectors; import org.apache.drill.common.exceptions.ExecutionSetupException; import org.apache.drill.common.util.DrillStringUtils; @@ -39,6 +41,7 @@ import org.apache.drill.exec.planner.PhysicalPlanReader; import org.apache.drill.exec.planner.fragment.Fragment.ExchangeFragmentPair; import org.apache.drill.exec.planner.fragment.Materializer.IndexedFragmentNode; +import org.apache.drill.common.DrillNode; import org.apache.drill.exec.proto.BitControl.Collector; import org.apache.drill.exec.proto.BitControl.PlanFragment; import org.apache.drill.exec.proto.BitControl.QueryContextInformation; @@ -175,7 +178,7 @@ public void collectStatsAndParallelizeFragments(PlanningSet planningSet, Set roots, - Collection activeEndpoints) throws PhysicalOperatorSetupException; + Map onlineEndpointUUIDs) throws ExecutionSetupException; /** * The starting function for the whole parallelization and memory computation logic. @@ -191,7 +194,7 @@ public abstract void adjustMemory(PlanningSet planningSet, Set roots, * @param options List of options set by the user. * @param foremanNode foreman node for this query plan. * @param queryId Query ID. - * @param activeEndpoints currently active endpoins on which this plan will run. + * @param onlineEndpoints currently active endpoins on which this plan will run. * @param rootFragment Root major fragment. * @param session session context. * @param queryContextInfo query context. @@ -200,17 +203,17 @@ public abstract void adjustMemory(PlanningSet planningSet, Set roots, */ @Override public final QueryWorkUnit generateWorkUnit(OptionList options, DrillbitEndpoint foremanNode, QueryId queryId, - Collection activeEndpoints, Fragment rootFragment, + Map onlineEndpoints, Fragment rootFragment, UserSession session, QueryContextInformation queryContextInfo) throws ExecutionSetupException { PlanningSet planningSet = prepareFragmentTree(rootFragment); Set rootFragments = getRootFragments(planningSet); - collectStatsAndParallelizeFragments(planningSet, rootFragments, activeEndpoints); + collectStatsAndParallelizeFragments(planningSet, rootFragments, onlineEndpoints.keySet()); - adjustMemory(planningSet, rootFragments, activeEndpoints); + adjustMemory(planningSet, rootFragments, onlineEndpoints); - return generateWorkUnit(options, foremanNode, queryId, rootFragment, planningSet, session, queryContextInfo); + return generateWorkUnit(options, foremanNode, queryId, rootFragment, planningSet, session, queryContextInfo, onlineEndpoints); } /** @@ -218,7 +221,7 @@ public final QueryWorkUnit generateWorkUnit(OptionList options, DrillbitEndpoint * @param options * @param foremanNode * @param queryId - * @param activeEndpoints + * @param onlineEndpointUUIDs * @param reader * @param rootFragment * @param session @@ -227,7 +230,7 @@ public final QueryWorkUnit generateWorkUnit(OptionList options, DrillbitEndpoint * @throws ExecutionSetupException */ public List getSplitFragments(OptionList options, DrillbitEndpoint foremanNode, QueryId queryId, - Collection activeEndpoints, PhysicalPlanReader reader, Fragment rootFragment, + Map onlineEndpointUUIDs, PhysicalPlanReader reader, Fragment rootFragment, UserSession session, QueryContextInformation queryContextInfo) throws ExecutionSetupException { // no op throw new UnsupportedOperationException("Use children classes"); @@ -293,8 +296,9 @@ protected void traverse(Wrapper fragmentWrapper, Consumer operation) th protected QueryWorkUnit generateWorkUnit(OptionList options, DrillbitEndpoint foremanNode, QueryId queryId, Fragment rootNode, PlanningSet planningSet, UserSession session, - QueryContextInformation queryContextInfo) throws ExecutionSetupException { + QueryContextInformation queryContextInfo, Map onlineEndpoints) throws ExecutionSetupException { List fragmentDefns = new ArrayList<>( ); + Map nodeMap = onlineEndpoints.entrySet().stream().collect(Collectors.toMap(entry -> DrillNode.create(entry.getKey()), entry -> entry.getValue())); MinorFragmentDefn rootFragmentDefn = null; FragmentRoot rootOperator = null; @@ -329,10 +333,13 @@ protected QueryWorkUnit generateWorkUnit(OptionList options, DrillbitEndpoint fo .setQueryId(queryId) .build(); + DrillbitEndpoint endpoint = wrapper.getAssignedEndpoint(minorFragmentId); + String endpointUUID = nodeMap.get(DrillNode.create(endpoint)); PlanFragment fragment = PlanFragment.newBuilder() .setForeman(foremanNode) .setHandle(handle) - .setAssignment(wrapper.getAssignedEndpoint(minorFragmentId)) + .setEndpointUUID(endpointUUID) + .setAssignment(endpoint) .setLeafFragment(isLeafFragment) .setContext(queryContextInfo) .setMemInitial(wrapper.getInitialAllocation()) 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 ffa577e41f5..fe8a46d791a 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 @@ -23,7 +23,7 @@ import java.util.function.Function; import java.util.stream.Collectors; -import org.apache.drill.exec.planner.cost.NodeResource; +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; @@ -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(); @@ -218,22 +218,22 @@ public List getFragmentDependencies() { */ public void computeCpuResources() { Preconditions.checkArgument(nodeResourceMap == null); - BinaryOperator merge = (first, second) -> { - NodeResource result = NodeResource.create(); + BinaryOperator merge = (first, second) -> { + NodeResources result = NodeResources.create(); result.add(first); result.add(second); return result; }; - Function cpuPerEndpoint = (endpoint) -> new NodeResource(1, 0); + Function cpuPerEndpoint = (endpoint) -> new NodeResources(1, 0); nodeResourceMap = endpoints.stream() .collect(Collectors.groupingBy(Function.identity(), - Collectors.reducing(NodeResource.create(), + 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/planner/fragment/ZKQueueParallelizer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/ZKQueueParallelizer.java index 28cd3256f92..6e529224b9d 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/ZKQueueParallelizer.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/ZKQueueParallelizer.java @@ -50,7 +50,7 @@ public ZKQueueParallelizer(boolean memoryAvailableInPlan, QueryResourceManager r @Override public void adjustMemory(PlanningSet planningSet, Set roots, - Collection activeEndpoints) throws PhysicalOperatorSetupException { + Map onlineEndpointUUIDs) throws PhysicalOperatorSetupException { if (planHasMemory) { return; } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/contrib/SplittingParallelizer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/contrib/SplittingParallelizer.java index 08aaca71840..1b865c6fec6 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/contrib/SplittingParallelizer.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/contrib/SplittingParallelizer.java @@ -17,11 +17,10 @@ */ package org.apache.drill.exec.planner.fragment.contrib; -import java.util.Collection; import java.util.Iterator; import java.util.List; import java.util.Set; - +import java.util.Map; import org.apache.drill.common.exceptions.ExecutionSetupException; import org.apache.drill.common.util.DrillStringUtils; import org.apache.drill.exec.ops.QueryContext; @@ -70,7 +69,7 @@ public SplittingParallelizer(boolean doMemoryPlanning, QueryContext context) { * @param options * @param foremanNode * @param queryId - * @param activeEndpoints + * @param onlineEndpointUUIDs * @param reader * @param rootFragment * @param session @@ -79,19 +78,18 @@ public SplittingParallelizer(boolean doMemoryPlanning, QueryContext context) { * @throws ExecutionSetupException */ public List getSplitFragments(OptionList options, DrillbitEndpoint foremanNode, QueryId queryId, - Collection activeEndpoints, PhysicalPlanReader reader, Fragment rootFragment, + Map onlineEndpointUUIDs, PhysicalPlanReader reader, Fragment rootFragment, UserSession session, QueryContextInformation queryContextInfo) throws ExecutionSetupException { final PlanningSet planningSet = this.prepareFragmentTree(rootFragment); Set rootFragments = getRootFragments(planningSet); - collectStatsAndParallelizeFragments(planningSet, rootFragments, activeEndpoints); + collectStatsAndParallelizeFragments(planningSet, rootFragments, onlineEndpointUUIDs.keySet()); - adjustMemory(planningSet, rootFragments, activeEndpoints); + adjustMemory(planningSet, rootFragments, onlineEndpointUUIDs); - return generateWorkUnits( - options, foremanNode, queryId, reader, rootFragment, planningSet, session, queryContextInfo); + return generateWorkUnits(options, foremanNode, queryId, reader, rootFragment, planningSet, session, queryContextInfo, onlineEndpointUUIDs); } /** @@ -113,7 +111,7 @@ public List getSplitFragments(OptionList options, DrillbitEndpoin */ private List generateWorkUnits(OptionList options, DrillbitEndpoint foremanNode, QueryId queryId, PhysicalPlanReader reader, Fragment rootNode, PlanningSet planningSet, - UserSession session, QueryContextInformation queryContextInfo) throws ExecutionSetupException { + UserSession session, QueryContextInformation queryContextInfo, Map onlineEndpointUUIDs) throws ExecutionSetupException { // now we generate all the individual plan fragments and associated assignments. Note, we need all endpoints // assigned before we can materialize, so we start a new loop here rather than utilizing the previous one. @@ -147,7 +145,7 @@ private List generateWorkUnits(OptionList options, DrillbitEndpoi DrillbitEndpoint[] endPoints = leafFragEndpoints; if ( plansCount == 0 ) { // no exchange, return list of single QueryWorkUnit - workUnits.add(generateWorkUnit(options, foremanNode, queryId, rootNode, planningSet, session, queryContextInfo)); + workUnits.add(generateWorkUnit(options, foremanNode, queryId, rootNode, planningSet, session, queryContextInfo, onlineEndpointUUIDs)); return workUnits; } 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 7b632e6ac49..76ab7f922df 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,6 +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 java.io.IOException; import java.util.Map; @@ -43,18 +44,18 @@ public class NodeResources { private long memoryInBytes; - private int numVirtualCpu; + private long numVirtualCpu; private static final int CURRENT_VERSION = 1; - public NodeResources(long memoryInBytes, int numVirtualCpu) { + public NodeResources(long memoryInBytes, long numVirtualCpu) { this(CURRENT_VERSION, memoryInBytes, numVirtualCpu); } @JsonCreator public NodeResources(@JsonProperty("version") int version, @JsonProperty("memoryInBytes") long memoryInBytes, - @JsonProperty("numVirtualCpu") int numVirtualCpu) { + @JsonProperty("numVirtualCpu") long numVirtualCpu) { this.version = version; this.memoryInBytes = memoryInBytes; this.numVirtualCpu = numVirtualCpu; @@ -78,7 +79,7 @@ public long getMemoryInGB() { return Math.round(getMemoryInMB() / 1024L); } - public int getNumVirtualCpu() { + public long getNumVirtualCpu() { return numVirtualCpu; } @@ -102,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; } @@ -116,7 +117,7 @@ public String toString() { @Override public int hashCode() { int result = 31 ^ Integer.hashCode(version); - result = result ^ Integer.hashCode(numVirtualCpu); + result = result ^ Long.hashCode(numVirtualCpu); result = result ^ Long.hashCode(memoryInBytes); return result; } @@ -135,6 +136,18 @@ public boolean equals(Object obj) { this.memoryInBytes == other.getMemoryInBytes(); } + public static NodeResources create() { + return create(0,0); + } + + public static NodeResources create(long cpu) { + return create(cpu,0); + } + + public static NodeResources create(long cpu, long memory) { + return new NodeResources(CURRENT_VERSION, memory, cpu); + } + public static class NodeResourcesDe extends StdDeserializer { private static final ObjectMapper mapper = new ObjectMapper(); 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 b2802de7118..bbcf0e42465 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 @@ -46,6 +46,7 @@ import org.apache.drill.exec.work.foreman.rm.ResourceManagerBuilder; import java.util.Collection; +import java.util.Map; import java.util.Map.Entry; import java.util.Set; import java.util.concurrent.ExecutorService; @@ -170,6 +171,10 @@ public Collection getBits() { return coord.getOnlineEndPoints(); } + public Map getOnlineEndpointUUIDs() { + return coord.getOnlineEndpointsUUID(); + } + public boolean isOnline(DrillbitEndpoint endpoint) { return endpoint.getState().equals(DrillbitEndpoint.State.ONLINE); } 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 479df1f8385..62e6846b8be 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 @@ -615,7 +615,8 @@ private QueryWorkUnit getQueryWorkUnit(final PhysicalPlan plan, return rm.getParallelizer(plan.getProperties().hasResourcePlan).generateWorkUnit(queryContext.getOptions().getOptionList(), queryContext.getCurrentEndpoint(), - queryId, queryContext.getOnlineEndpoints(), + queryId, + queryContext.getOnlineEndpointUUIDs(), rootFragment, initiatingClient.getSession(), queryContext.getQueryContextInfo()); } 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 48b9d81487a..5b4c4f9a4d3 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 @@ -216,7 +216,7 @@ public long minimumOperatorMemory() { @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); + return new DistributedQueueParallelizer(true || planHasMemory, this.context, this); } @Override diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/user/PlanSplitter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/user/PlanSplitter.java index 66f4dfd27bb..9a853899a11 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/user/PlanSplitter.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/user/PlanSplitter.java @@ -115,7 +115,7 @@ private List getFragments(final DrillbitContext dContext, final Ge if ( req.getSplitPlan() ) { final List queryWorkUnits = parallelizer.getSplitFragments( queryContext.getOptions().getOptionList(), queryContext.getCurrentEndpoint(), - queryId, queryContext.getActiveEndpoints(), dContext.getPlanReader(), rootFragment, + queryId, queryContext.getOnlineEndpointUUIDs(), dContext.getPlanReader(), rootFragment, queryContext.getSession(), queryContext.getQueryContextInfo()); for (QueryWorkUnit queryWorkUnit : queryWorkUnits) { @@ -130,7 +130,7 @@ private List getFragments(final DrillbitContext dContext, final Ge } } else { final QueryWorkUnit queryWorkUnit = parallelizer.generateWorkUnit(queryContext.getOptions().getOptionList(), queryContext.getCurrentEndpoint(), - queryId, queryContext.getActiveEndpoints(), rootFragment, + queryId, queryContext.getOnlineEndpointUUIDs(), rootFragment, queryContext.getSession(), queryContext.getQueryContextInfo()); // planner.visitPhysicalPlan(queryWorkUnit); queryWorkUnit.applyPlan(dContext.getPlanReader()); diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestLocalExchange.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestLocalExchange.java index 6e508ac031a..90d35a958bd 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestLocalExchange.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestLocalExchange.java @@ -399,7 +399,7 @@ private static void testHelperVerifyPartitionSenderParallelization( final QueryContextInformation queryContextInfo = Utilities.createQueryContextInfo("dummySchemaName", "938ea2d9-7cb9-4baf-9414-a5a0b7777e8e"); QueryWorkUnit qwu = PARALLELIZER.generateWorkUnit(new OptionList(), drillbitContext.getEndpoint(), QueryId.getDefaultInstance(), - drillbitContext.getBits(), rootFragment, USER_SESSION, queryContextInfo); + drillbitContext.getOnlineEndpointUUIDs(), rootFragment, USER_SESSION, queryContextInfo); qwu.applyPlan(planReader); // Make sure the number of minor fragments with HashPartitioner within a major fragment is not more than the diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/partitionsender/TestPartitionSender.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/partitionsender/TestPartitionSender.java index c246eba0069..e8724cb9110 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/partitionsender/TestPartitionSender.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/partitionsender/TestPartitionSender.java @@ -213,7 +213,7 @@ private void testThreadsHelper(HashToRandomExchange hashToRandomExchange, Drillb final QueryContextInformation queryContextInfo = Utilities.createQueryContextInfo("dummySchemaName", "938ea2d9-7cb9-4baf-9414-a5a0b7777e8e"); final QueryWorkUnit qwu = PARALLELIZER.generateWorkUnit(options, drillbitContext.getEndpoint(), QueryId.getDefaultInstance(), - drillbitContext.getBits(), rootFragment, USER_SESSION, queryContextInfo); + drillbitContext.getOnlineEndpointUUIDs(), rootFragment, USER_SESSION, queryContextInfo); qwu.applyPlan(planReader); final List mfEndPoints = PhysicalOperatorUtil.getIndexOrderedEndpoints(Lists.newArrayList(drillbitContext.getBits())); 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 198a5290118..fd3d454f78a 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,7 +22,6 @@ 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.cost.NodeResource; import org.apache.drill.exec.planner.fragment.Fragment; import org.apache.drill.exec.planner.fragment.PlanningSet; import org.apache.drill.exec.planner.fragment.DistributedQueueParallelizer; @@ -32,8 +31,12 @@ import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint; import org.apache.drill.exec.proto.UserBitShared; import org.apache.drill.exec.proto.UserProtos; +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.rpc.user.UserSession; import org.apache.drill.exec.server.DrillbitContext; +import org.apache.drill.exec.work.foreman.rm.QueryResourceManager; import org.apache.drill.exec.work.foreman.rm.EmbeddedQueryQueue; import org.apache.drill.shaded.guava.com.google.common.collect.Iterables; import org.apache.drill.test.ClientFixture; @@ -88,8 +91,18 @@ public static void close() throws Exception { queryContext.close(); } + private QueryResourceManager mockResourceManager() throws QueueSelectionException { + final QueryResourceManager mockRM = mock(QueryResourceManager.class); + final QueryQueueConfig queueConfig = mock(QueryQueueConfig.class); + + when(queueConfig.getMaxQueryMemoryInMBPerNode()).thenReturn(10L); + when(mockRM.selectQueue(any(NodeResources.class))).thenReturn(queueConfig); + + return mockRM; + } + private final Wrapper mockWrapper(Wrapper rootFragment, - Map resourceMap, + Map resourceMap, List endpoints, Map originalToMockWrapper ) { final Wrapper mockWrapper = mock(Wrapper.class); @@ -110,7 +123,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, @@ -147,15 +160,16 @@ private String getPlanForQuery(String query, long outputBatchSize, return plan; } - private List getEndpoints(int totalMinorFragments, - Set notIn) { - List endpoints = new ArrayList<>(); + private Map getEndpoints(int totalMinorFragments, + Set notIn) { + Map endpoints = new HashMap<>(); Iterator drillbits = Iterables.cycle(nodeList).iterator(); + int i=0; while(totalMinorFragments-- > 0) { DrillbitEndpoint dbit = (DrillbitEndpoint) drillbits.next(); if (!notIn.contains(dbit)) { - endpoints.add(dbit); + endpoints.put(dbit, "drillbit" + ++i); } } return endpoints; @@ -176,7 +190,7 @@ 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); @@ -184,46 +198,46 @@ private PlanningSet preparePlanningSet(List activeEndpoints, l @Test public void TestSingleMajorFragmentWithProjectAndScan() throws Exception { - List activeEndpoints = getEndpoints(2, new HashSet<>()); - Map resources = activeEndpoints.stream() + Map onlineEndpoints = getEndpoints(2, new HashSet<>()); + Map resources = onlineEndpoints.keySet().stream() .collect(Collectors.toMap(x -> x, - x -> NodeResource.create())); + x -> NodeResources.create())); String sql = "SELECT * from cp.`tpch/nation.parquet`"; - SimpleParallelizer parallelizer = new DistributedQueueParallelizer(false, queryContext); - PlanningSet planningSet = preparePlanningSet(activeEndpoints, DEFAULT_SLICE_TARGET, resources, sql, parallelizer); - parallelizer.adjustMemory(planningSet, createSet(planningSet.getRootWrapper()), activeEndpoints); - assertTrue("memory requirement is different", Iterables.all(resources.entrySet(), (e) -> e.getValue().getMemory() == 30)); + SimpleParallelizer parallelizer = new DistributedQueueParallelizer(false, queryContext, mockResourceManager()); + PlanningSet planningSet = preparePlanningSet(new ArrayList<>(onlineEndpoints.keySet()), DEFAULT_SLICE_TARGET, resources, sql, parallelizer); + parallelizer.adjustMemory(planningSet, createSet(planningSet.getRootWrapper()), onlineEndpoints); + assertTrue("memory requirement is different", Iterables.all(resources.entrySet(), (e) -> e.getValue().getMemoryInBytes() == 30)); } @Test public void TestSingleMajorFragmentWithGroupByProjectAndScan() throws Exception { - List activeEndpoints = getEndpoints(2, new HashSet<>()); - Map resources = activeEndpoints.stream() + Map onlineEndpoints = getEndpoints(2, new HashSet<>()); + Map resources = onlineEndpoints.keySet().stream() .collect(Collectors.toMap(x -> x, - x -> NodeResource.create())); + x -> NodeResources.create())); String sql = "SELECT dept_id, count(*) from cp.`tpch/lineitem.parquet` group by dept_id"; - SimpleParallelizer parallelizer = new DistributedQueueParallelizer(false, queryContext); - PlanningSet planningSet = preparePlanningSet(activeEndpoints, DEFAULT_SLICE_TARGET, resources, sql, parallelizer); - parallelizer.adjustMemory(planningSet, createSet(planningSet.getRootWrapper()), activeEndpoints); - assertTrue("memory requirement is different", Iterables.all(resources.entrySet(), (e) -> e.getValue().getMemory() == 529570)); + SimpleParallelizer parallelizer = new DistributedQueueParallelizer(false, queryContext, mockResourceManager()); + PlanningSet planningSet = preparePlanningSet(new ArrayList<>(onlineEndpoints.keySet()), DEFAULT_SLICE_TARGET, resources, sql, parallelizer); + parallelizer.adjustMemory(planningSet, createSet(planningSet.getRootWrapper()), onlineEndpoints); + assertTrue("memory requirement is different", Iterables.all(resources.entrySet(), (e) -> e.getValue().getMemoryInBytes() == 529570)); } @Test public void TestTwoMajorFragmentWithSortyProjectAndScan() throws Exception { - List activeEndpoints = getEndpoints(2, new HashSet<>()); - Map resources = activeEndpoints.stream() + Map onlineEndpoints = getEndpoints(2, new HashSet<>()); + Map resources = onlineEndpoints.keySet().stream() .collect(Collectors.toMap(x -> x, - x -> NodeResource.create())); + x -> NodeResources.create())); String sql = "SELECT * from cp.`tpch/lineitem.parquet` order by dept_id"; - SimpleParallelizer parallelizer = new DistributedQueueParallelizer(false, queryContext); - PlanningSet planningSet = preparePlanningSet(activeEndpoints, 2, resources, sql, parallelizer); - parallelizer.adjustMemory(planningSet, createSet(planningSet.getRootWrapper()), activeEndpoints); - assertTrue("memory requirement is different", Iterables.all(resources.entrySet(), (e) -> e.getValue().getMemory() == 481490)); + SimpleParallelizer parallelizer = new DistributedQueueParallelizer(false, queryContext, mockResourceManager()); + 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)); } @Test 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 f3b15884c45..0c174d4a3be 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,7 +17,8 @@ */ package org.apache.drill.exec.pop; -import java.util.List; +import java.util.Map; +import java.util.HashMap; import org.apache.drill.categories.PlannerTest; import org.apache.drill.exec.planner.PhysicalPlanReader; @@ -35,7 +36,6 @@ import org.apache.drill.exec.work.QueryWorkUnit; import org.junit.Test; -import org.apache.drill.shaded.guava.com.google.common.collect.Lists; import org.junit.experimental.categories.Category; import static org.junit.Assert.assertEquals; @@ -54,14 +54,15 @@ private void print(String fragmentFile, int bitCount, int expectedFragmentCount) PhysicalPlanReader ppr = PhysicalPlanReaderTestFactory.defaultPhysicalPlanReader(CONFIG); Fragment fragmentRoot = getRootFragment(ppr, fragmentFile); SimpleParallelizer par = new DefaultParallelizer(true, 1000*1000, 5, 10, 1.2); - List endpoints = Lists.newArrayList(); + Map endpoints = new HashMap<>(); DrillbitEndpoint localBit = null; for(int i =0; i < bitCount; i++) { DrillbitEndpoint b1 = DrillbitEndpoint.newBuilder().setAddress("localhost").setControlPort(1234+i).build(); if (i == 0) { localBit = b1; } - endpoints.add(b1); + StringBuilder sb = new StringBuilder(); + endpoints.put(b1, sb.append("Drillbit-").append(i).toString()); } final QueryContextInformation queryContextInfo = Utilities.createQueryContextInfo("dummySchemaName", "938ea2d9-7cb9-4baf-9414-a5a0b7777e8e"); diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/BitControl.java b/protocol/src/main/java/org/apache/drill/exec/proto/BitControl.java index 3bf0c0ad44b..63ce995d1e5 100644 --- a/protocol/src/main/java/org/apache/drill/exec/proto/BitControl.java +++ b/protocol/src/main/java/org/apache/drill/exec/proto/BitControl.java @@ -4491,6 +4491,20 @@ public interface PlanFragmentOrBuilder extends */ org.apache.drill.exec.proto.BitControl.CollectorOrBuilder getCollectorOrBuilder( int index); + + /** + * optional string endpointUUID = 18; + */ + boolean hasEndpointUUID(); + /** + * optional string endpointUUID = 18; + */ + java.lang.String getEndpointUUID(); + /** + * optional string endpointUUID = 18; + */ + com.google.protobuf.ByteString + getEndpointUUIDBytes(); } /** * Protobuf type {@code exec.bit.control.PlanFragment} @@ -4515,6 +4529,7 @@ private PlanFragment() { memMax_ = 2000000000L; optionsJson_ = ""; collector_ = java.util.Collections.emptyList(); + endpointUUID_ = ""; } @java.lang.Override @@ -4662,6 +4677,12 @@ private PlanFragment( input.readMessage(org.apache.drill.exec.proto.BitControl.Collector.PARSER, extensionRegistry)); break; } + case 146: { + com.google.protobuf.ByteString bs = input.readBytes(); + bitField0_ |= 0x00004000; + endpointUUID_ = bs; + break; + } default: { if (!parseUnknownField( input, unknownFields, extensionRegistry, tag)) { @@ -5043,6 +5064,48 @@ public org.apache.drill.exec.proto.BitControl.CollectorOrBuilder getCollectorOrB return collector_.get(index); } + public static final int ENDPOINTUUID_FIELD_NUMBER = 18; + private volatile java.lang.Object endpointUUID_; + /** + * optional string endpointUUID = 18; + */ + public boolean hasEndpointUUID() { + return ((bitField0_ & 0x00004000) == 0x00004000); + } + /** + * optional string endpointUUID = 18; + */ + public java.lang.String getEndpointUUID() { + java.lang.Object ref = endpointUUID_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + endpointUUID_ = s; + } + return s; + } + } + /** + * optional string endpointUUID = 18; + */ + public com.google.protobuf.ByteString + getEndpointUUIDBytes() { + java.lang.Object ref = endpointUUID_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + endpointUUID_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + private byte memoizedIsInitialized = -1; @java.lang.Override public final boolean isInitialized() { @@ -5102,6 +5165,9 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) for (int i = 0; i < collector_.size(); i++) { output.writeMessage(17, collector_.get(i)); } + if (((bitField0_ & 0x00004000) == 0x00004000)) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 18, endpointUUID_); + } unknownFields.writeTo(output); } @@ -5169,6 +5235,9 @@ public int getSerializedSize() { size += com.google.protobuf.CodedOutputStream .computeMessageSize(17, collector_.get(i)); } + if (((bitField0_ & 0x00004000) == 0x00004000)) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(18, endpointUUID_); + } size += unknownFields.getSerializedSize(); memoizedSize = size; return size; @@ -5265,6 +5334,11 @@ public boolean equals(final java.lang.Object obj) { } result = result && getCollectorList() .equals(other.getCollectorList()); + result = result && (hasEndpointUUID() == other.hasEndpointUUID()); + if (hasEndpointUUID()) { + result = result && getEndpointUUID() + .equals(other.getEndpointUUID()); + } result = result && unknownFields.equals(other.unknownFields); return result; } @@ -5343,6 +5417,10 @@ public int hashCode() { hash = (37 * hash) + COLLECTOR_FIELD_NUMBER; hash = (53 * hash) + getCollectorList().hashCode(); } + if (hasEndpointUUID()) { + hash = (37 * hash) + ENDPOINTUUID_FIELD_NUMBER; + hash = (53 * hash) + getEndpointUUID().hashCode(); + } hash = (29 * hash) + unknownFields.hashCode(); memoizedHashCode = hash; return hash; @@ -5536,6 +5614,8 @@ public Builder clear() { } else { collectorBuilder_.clear(); } + endpointUUID_ = ""; + bitField0_ = (bitField0_ & ~0x00008000); return this; } @@ -5649,6 +5729,10 @@ public org.apache.drill.exec.proto.BitControl.PlanFragment buildPartial() { } else { result.collector_ = collectorBuilder_.build(); } + if (((from_bitField0_ & 0x00008000) == 0x00008000)) { + to_bitField0_ |= 0x00004000; + } + result.endpointUUID_ = endpointUUID_; result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -5770,6 +5854,11 @@ public Builder mergeFrom(org.apache.drill.exec.proto.BitControl.PlanFragment oth } } } + if (other.hasEndpointUUID()) { + bitField0_ |= 0x00008000; + endpointUUID_ = other.endpointUUID_; + onChanged(); + } this.mergeUnknownFields(other.unknownFields); onChanged(); return this; @@ -7037,6 +7126,82 @@ public org.apache.drill.exec.proto.BitControl.Collector.Builder addCollectorBuil } return collectorBuilder_; } + + private java.lang.Object endpointUUID_ = ""; + /** + * optional string endpointUUID = 18; + */ + public boolean hasEndpointUUID() { + return ((bitField0_ & 0x00008000) == 0x00008000); + } + /** + * optional string endpointUUID = 18; + */ + public java.lang.String getEndpointUUID() { + java.lang.Object ref = endpointUUID_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + endpointUUID_ = s; + } + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * optional string endpointUUID = 18; + */ + public com.google.protobuf.ByteString + getEndpointUUIDBytes() { + java.lang.Object ref = endpointUUID_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + endpointUUID_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * optional string endpointUUID = 18; + */ + public Builder setEndpointUUID( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00008000; + endpointUUID_ = value; + onChanged(); + return this; + } + /** + * optional string endpointUUID = 18; + */ + public Builder clearEndpointUUID() { + bitField0_ = (bitField0_ & ~0x00008000); + endpointUUID_ = getDefaultInstance().getEndpointUUID(); + onChanged(); + return this; + } + /** + * optional string endpointUUID = 18; + */ + public Builder setEndpointUUIDBytes( + com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00008000; + endpointUUID_ = value; + onChanged(); + return this; + } @java.lang.Override public final Builder setUnknownFields( final com.google.protobuf.UnknownFieldSet unknownFields) { @@ -10707,7 +10872,7 @@ public org.apache.drill.exec.proto.BitControl.FinishedReceiver getDefaultInstanc "c.bit.FragmentHandle\"G\n\023InitializeFragme" + "nts\0220\n\010fragment\030\001 \003(\0132\036.exec.bit.control" + ".PlanFragment\".\n\rCustomMessage\022\014\n\004type\030\001" + - " \001(\005\022\017\n\007message\030\002 \001(\014\"\374\003\n\014PlanFragment\022(" + + " \001(\005\022\017\n\007message\030\002 \001(\014\"\222\004\n\014PlanFragment\022(" + "\n\006handle\030\001 \001(\0132\030.exec.bit.FragmentHandle" + "\022\024\n\014network_cost\030\004 \001(\002\022\020\n\010cpu_cost\030\005 \001(\002" + "\022\021\n\tdisk_cost\030\006 \001(\002\022\023\n\013memory_cost\030\007 \001(\002" + @@ -10720,29 +10885,30 @@ public org.apache.drill.exec.proto.BitControl.FinishedReceiver getDefaultInstanc "ls\022\024\n\014options_json\030\017 \001(\t\022:\n\007context\030\020 \001(" + "\0132).exec.bit.control.QueryContextInforma" + "tion\022.\n\tcollector\030\021 \003(\0132\033.exec.bit.contr" + - "ol.Collector\"\210\001\n\tCollector\022\"\n\032opposite_m" + - "ajor_fragment_id\030\001 \001(\005\022#\n\027incoming_minor" + - "_fragment\030\002 \003(\005B\002\020\001\022\035\n\025supports_out_of_o" + - "rder\030\003 \001(\010\022\023\n\013is_spooling\030\004 \001(\010\"w\n\027Query" + - "ContextInformation\022\030\n\020query_start_time\030\001" + - " \001(\003\022\021\n\ttime_zone\030\002 \001(\005\022\033\n\023default_schem" + - "a_name\030\003 \001(\t\022\022\n\nsession_id\030\004 \001(\t\"f\n\017Work" + - "QueueStatus\022(\n\010endpoint\030\001 \001(\0132\026.exec.Dri" + - "llbitEndpoint\022\024\n\014queue_length\030\002 \001(\005\022\023\n\013r" + - "eport_time\030\003 \001(\003\"h\n\020FinishedReceiver\022*\n\010" + - "receiver\030\001 \001(\0132\030.exec.bit.FragmentHandle" + - "\022(\n\006sender\030\002 \001(\0132\030.exec.bit.FragmentHand" + - "le*\206\003\n\007RpcType\022\r\n\tHANDSHAKE\020\000\022\007\n\003ACK\020\001\022\013" + - "\n\007GOODBYE\020\002\022\034\n\030REQ_INITIALIZE_FRAGMENTS\020" + - "\003\022\027\n\023REQ_CANCEL_FRAGMENT\020\006\022\031\n\025REQ_RECEIV" + - "ER_FINISHED\020\007\022\027\n\023REQ_FRAGMENT_STATUS\020\010\022\022" + - "\n\016REQ_BIT_STATUS\020\t\022\024\n\020REQ_QUERY_STATUS\020\n" + - "\022\024\n\020REQ_QUERY_CANCEL\020\017\022\030\n\024REQ_UNPAUSE_FR" + - "AGMENT\020\020\022\016\n\nREQ_CUSTOM\020\021\022\030\n\024RESP_FRAGMEN" + - "T_HANDLE\020\013\022\030\n\024RESP_FRAGMENT_STATUS\020\014\022\023\n\017" + - "RESP_BIT_STATUS\020\r\022\025\n\021RESP_QUERY_STATUS\020\016" + - "\022\017\n\013RESP_CUSTOM\020\022\022\020\n\014SASL_MESSAGE\020\023B+\n\033o" + - "rg.apache.drill.exec.protoB\nBitControlH\001" + "ol.Collector\022\024\n\014endpointUUID\030\022 \001(\t\"\210\001\n\tC" + + "ollector\022\"\n\032opposite_major_fragment_id\030\001" + + " \001(\005\022#\n\027incoming_minor_fragment\030\002 \003(\005B\002\020" + + "\001\022\035\n\025supports_out_of_order\030\003 \001(\010\022\023\n\013is_s" + + "pooling\030\004 \001(\010\"w\n\027QueryContextInformation" + + "\022\030\n\020query_start_time\030\001 \001(\003\022\021\n\ttime_zone\030" + + "\002 \001(\005\022\033\n\023default_schema_name\030\003 \001(\t\022\022\n\nse" + + "ssion_id\030\004 \001(\t\"f\n\017WorkQueueStatus\022(\n\010end" + + "point\030\001 \001(\0132\026.exec.DrillbitEndpoint\022\024\n\014q" + + "ueue_length\030\002 \001(\005\022\023\n\013report_time\030\003 \001(\003\"h" + + "\n\020FinishedReceiver\022*\n\010receiver\030\001 \001(\0132\030.e" + + "xec.bit.FragmentHandle\022(\n\006sender\030\002 \001(\0132\030" + + ".exec.bit.FragmentHandle*\206\003\n\007RpcType\022\r\n\t" + + "HANDSHAKE\020\000\022\007\n\003ACK\020\001\022\013\n\007GOODBYE\020\002\022\034\n\030REQ" + + "_INITIALIZE_FRAGMENTS\020\003\022\027\n\023REQ_CANCEL_FR" + + "AGMENT\020\006\022\031\n\025REQ_RECEIVER_FINISHED\020\007\022\027\n\023R" + + "EQ_FRAGMENT_STATUS\020\010\022\022\n\016REQ_BIT_STATUS\020\t" + + "\022\024\n\020REQ_QUERY_STATUS\020\n\022\024\n\020REQ_QUERY_CANC" + + "EL\020\017\022\030\n\024REQ_UNPAUSE_FRAGMENT\020\020\022\016\n\nREQ_CU" + + "STOM\020\021\022\030\n\024RESP_FRAGMENT_HANDLE\020\013\022\030\n\024RESP" + + "_FRAGMENT_STATUS\020\014\022\023\n\017RESP_BIT_STATUS\020\r\022" + + "\025\n\021RESP_QUERY_STATUS\020\016\022\017\n\013RESP_CUSTOM\020\022\022" + + "\020\n\014SASL_MESSAGE\020\023B+\n\033org.apache.drill.ex" + + "ec.protoB\nBitControlH\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor. InternalDescriptorAssigner() { @@ -10794,7 +10960,7 @@ public com.google.protobuf.ExtensionRegistry assignDescriptors( internal_static_exec_bit_control_PlanFragment_fieldAccessorTable = new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( internal_static_exec_bit_control_PlanFragment_descriptor, - new java.lang.String[] { "Handle", "NetworkCost", "CpuCost", "DiskCost", "MemoryCost", "FragmentJson", "LeafFragment", "Assignment", "Foreman", "MemInitial", "MemMax", "Credentials", "OptionsJson", "Context", "Collector", }); + new java.lang.String[] { "Handle", "NetworkCost", "CpuCost", "DiskCost", "MemoryCost", "FragmentJson", "LeafFragment", "Assignment", "Foreman", "MemInitial", "MemMax", "Credentials", "OptionsJson", "Context", "Collector", "EndpointUUID", }); internal_static_exec_bit_control_Collector_descriptor = getDescriptor().getMessageTypes().get(6); internal_static_exec_bit_control_Collector_fieldAccessorTable = new diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/SchemaBitControl.java b/protocol/src/main/java/org/apache/drill/exec/proto/SchemaBitControl.java index 475419586ac..735549f1d5b 100644 --- a/protocol/src/main/java/org/apache/drill/exec/proto/SchemaBitControl.java +++ b/protocol/src/main/java/org/apache/drill/exec/proto/SchemaBitControl.java @@ -672,6 +672,8 @@ public void writeTo(com.dyuproject.protostuff.Output output, org.apache.drill.ex for(org.apache.drill.exec.proto.BitControl.Collector collector : message.getCollectorList()) output.writeObject(17, collector, org.apache.drill.exec.proto.SchemaBitControl.Collector.WRITE, true); + if(message.hasEndpointUUID()) + output.writeString(18, message.getEndpointUUID(), false); } public boolean isInitialized(org.apache.drill.exec.proto.BitControl.PlanFragment message) { @@ -761,6 +763,9 @@ public void mergeFrom(com.dyuproject.protostuff.Input input, org.apache.drill.ex case 17: builder.addCollector(input.mergeObject(org.apache.drill.exec.proto.BitControl.Collector.newBuilder(), org.apache.drill.exec.proto.SchemaBitControl.Collector.MERGE)); + break; + case 18: + builder.setEndpointUUID(input.readString()); break; default: input.handleUnknownField(number, this); @@ -817,6 +822,7 @@ public static java.lang.String getFieldName(int number) case 15: return "optionsJson"; case 16: return "context"; case 17: return "collector"; + case 18: return "endpointUUID"; default: return null; } } @@ -843,6 +849,7 @@ public static int getFieldNumber(java.lang.String name) fieldMap.put("optionsJson", 15); fieldMap.put("context", 16); fieldMap.put("collector", 17); + fieldMap.put("endpointUUID", 18); } } diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/beans/PlanFragment.java b/protocol/src/main/java/org/apache/drill/exec/proto/beans/PlanFragment.java index ebbdf636805..7214f15882a 100644 --- a/protocol/src/main/java/org/apache/drill/exec/proto/beans/PlanFragment.java +++ b/protocol/src/main/java/org/apache/drill/exec/proto/beans/PlanFragment.java @@ -66,6 +66,7 @@ public static PlanFragment getDefaultInstance() private String optionsJson; private QueryContextInformation context; private List collector; + private String endpointUUID; public PlanFragment() { @@ -269,6 +270,19 @@ public PlanFragment setCollectorList(List collector) return this; } + // endpointUUID + + public String getEndpointUUID() + { + return endpointUUID; + } + + public PlanFragment setEndpointUUID(String endpointUUID) + { + this.endpointUUID = endpointUUID; + return this; + } + // java serialization public void readExternal(ObjectInput in) throws IOException @@ -376,6 +390,9 @@ public void mergeFrom(Input input, PlanFragment message) throws IOException message.collector.add(input.mergeObject(null, Collector.getSchema())); break; + case 18: + message.endpointUUID = input.readString(); + break; default: input.handleUnknownField(number, this); } @@ -441,6 +458,9 @@ public void writeTo(Output output, PlanFragment message) throws IOException } } + + if(message.endpointUUID != null) + output.writeString(18, message.endpointUUID, false); } public String getFieldName(int number) @@ -462,6 +482,7 @@ public String getFieldName(int number) case 15: return "optionsJson"; case 16: return "context"; case 17: return "collector"; + case 18: return "endpointUUID"; default: return null; } } @@ -490,6 +511,7 @@ public int getFieldNumber(String name) __fieldMap.put("optionsJson", 15); __fieldMap.put("context", 16); __fieldMap.put("collector", 17); + __fieldMap.put("endpointUUID", 18); } } diff --git a/protocol/src/main/protobuf/BitControl.proto b/protocol/src/main/protobuf/BitControl.proto index 3bc3c86538a..a30b0606017 100644 --- a/protocol/src/main/protobuf/BitControl.proto +++ b/protocol/src/main/protobuf/BitControl.proto @@ -79,6 +79,7 @@ message PlanFragment { optional string options_json = 15; optional QueryContextInformation context = 16; repeated Collector collector = 17; + optional string endpointUUID = 18; } message Collector { From 2800c57981f27d04b97e5994c3f6325ca301f110 Mon Sep 17 00:00:00 2001 From: Sorabh Hamirwasia Date: Fri, 22 Mar 2019 16:43:25 -0700 Subject: [PATCH 5/7] DRILL-7191: RM blobs persistence in Zookeeper for Distributed RM. 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 | 361 ++++++++++---- .../resourcemgr/TestDistributedQueryRM.java | 166 +++++++ .../resourcemgr/TestE2EWithDistributedRM.java | 59 +++ .../exec/resourcemgr/TestRMConfigLoad.java | 35 +- .../org/apache/drill/test/ExampleTest.java | 17 +- .../apache/drill/exec/memory/Accountant.java | 12 +- 45 files changed, 1544 insertions(+), 575 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..e09c32eaf4e 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; @@ -34,17 +40,22 @@ import org.apache.drill.test.ClusterFixture; import org.apache.drill.test.ClusterFixtureBuilder; import org.apache.drill.test.DrillTest; +import org.junit.After; import org.junit.Before; import org.junit.Rule; import org.junit.Test; 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 +66,291 @@ 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.RM_ENABLED, false) + .setOptionDefault(ExecConstants.ENABLE_QUEUE.getOptionName(), false) + .configProperty(ExecConstants.DRILL_PORT_HUNT, true) + .configProperty(ExecConstants.ALLOW_LOOPBACK_ADDRESS_BINDING, true) + .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); + } + + @After + public void testCleanup() throws Exception { + cluster.close(); + } + + 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..c9c76100cbc --- /dev/null +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/resourcemgr/TestDistributedQueryRM.java @@ -0,0 +1,166 @@ +/* + * 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.common.DrillNode; +import org.apache.drill.exec.ExecConstants; +import org.apache.drill.exec.ops.QueryContext; +import org.apache.drill.exec.proto.CoordinationProtos; +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 java.util.stream.Collectors; + +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) + .configProperty(ExecConstants.ALLOW_LOOPBACK_ADDRESS_BINDING, true) + .withLocalZk(); + startCluster(fixtureBuilder); + context = cluster.drillbit().getContext(); + final ResourceManager rm = context.getResourceManager(); + Preconditions.checkState(rm instanceof DistributedResourceManager); + drillRM = (DistributedResourceManager) rm; + + final Map onlineEndpoints = context.getOnlineEndpointUUIDs(); + final Map onlineEndpointNodes = onlineEndpoints.entrySet().stream() + .collect(Collectors.toMap(x -> DrillNode.create(x.getKey()), x -> x.getValue())); + + 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.getOnlineEndpointNodeUUIDs()).thenReturn(onlineEndpointNodes); + when(queryContext.getCurrentEndpoint()).thenReturn(context.getEndpoint()); + when(queryContext.getCurrentEndpointNode()).thenReturn(DrillNode.create(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..80b0dfe9526 --- /dev/null +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/resourcemgr/TestE2EWithDistributedRM.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.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) + .setOptionDefault(ExecConstants.ENABLE_QUEUE.getOptionName(), false) + .configProperty(ExecConstants.DRILL_PORT_HUNT, true) + .configProperty(ExecConstants.ALLOW_LOOPBACK_ADDRESS_BINDING, true) + .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..a58e86f40b6 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,12 +25,13 @@ 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; import org.apache.drill.test.DrillTest; -import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -38,7 +39,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -@Ignore("These tests will be ignored until integration with new DistributedResourceManager is done") @Category(ResourceManagerTest.class) public final class TestRMConfigLoad extends DrillTest { @@ -49,10 +49,12 @@ public final class TestRMConfigLoad extends DrillTest { public void testDefaultRMConfig() throws Exception { ClusterFixtureBuilder fixtureBuilder = ClusterFixture.builder(dirTestWatcher) .configProperty(ExecConstants.RM_ENABLED, true) + .setOptionDefault(ExecConstants.ENABLE_QUEUE.getOptionName(), false) + .configProperty(ExecConstants.ALLOW_LOOPBACK_ADDRESS_BINDING, true) .configProperty(ExecConstants.DRILL_PORT_HUNT, true) .withLocalZk(); - try (ClusterFixture cluster = fixtureBuilder.build()) { + try(ClusterFixture cluster = fixtureBuilder.build()) { ResourceManager resourceManager = cluster.drillbit().getContext().getResourceManager(); assertTrue(resourceManager instanceof DistributedResourceManager); @@ -83,7 +85,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 +97,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(); @@ -106,12 +110,31 @@ public void testDefaultRMWithLocalCoordinatorAndRMDisabled() throws Exception { public void testDefaultRMOnlyRMDisabled() throws Exception { ClusterFixtureBuilder fixtureBuilder = ClusterFixture.builder(dirTestWatcher) .configProperty(ExecConstants.RM_ENABLED, false) + .setOptionDefault(ExecConstants.ENABLE_QUEUE.getOptionName(), false) + .configProperty(ExecConstants.ALLOW_LOOPBACK_ADDRESS_BINDING, true) .configProperty(ExecConstants.DRILL_PORT_HUNT, true) .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) + .setOptionDefault(ExecConstants.ENABLE_QUEUE.getOptionName(), true) + .configProperty(ExecConstants.DRILL_PORT_HUNT, true) + .configProperty(ExecConstants.ALLOW_LOOPBACK_ADDRESS_BINDING, 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; From 1517a87eb1effb2aac0c75b5f5ea6abc25407ab0 Mon Sep 17 00:00:00 2001 From: HanumathRao Date: Thu, 18 Apr 2019 15:44:30 -0700 Subject: [PATCH 6/7] DRILL-7193: Integration changes of the Distributed RM queue configuration with Simple Parallelizer. Changes to set the memory allocation per operator in query profile. Addressing an memory minimization logic was not considering non-buffered operators. Handling error cases when memory requirements for buffered or non-buffered cannot be reduced. --- .../client/src/protobuf/BitControl.pb.cc | 231 +++++++++++------- .../client/src/protobuf/BitControl.pb.h | 156 +++++++++--- .../client/src/protobuf/UserBitShared.pb.cc | 177 ++++++++------ .../client/src/protobuf/UserBitShared.pb.h | 34 +++ .../apache/drill/exec/ops/OpProfileDef.java | 4 +- .../drill/exec/ops/OperatorContextImpl.java | 2 +- .../apache/drill/exec/ops/OperatorStats.java | 10 +- .../exec/physical/base/AbstractGroupScan.java | 12 - .../exec/physical/impl/BaseRootExec.java | 2 +- .../DistributedQueueParallelizer.java | 116 ++++++--- .../drill/exec/planner/fragment/Fragment.java | 10 +- .../exec/planner/fragment/Materializer.java | 10 +- .../planner/fragment/MemoryCalculator.java | 24 +- .../planner/fragment/ZKQueueParallelizer.java | 1 - .../drill/exec/planner/sql/DirectPlan.java | 15 +- .../sql/handlers/ShowFilesHandler.java | 2 +- .../RMConsistentBlobStoreManager.java | 2 +- .../drill/exec/work/foreman/Foreman.java | 2 +- .../rm/DistributedResourceManager.java | 1 - .../drill/exec/memory/TestAllocators.java | 6 +- .../managed/TestExternalSortInternals.java | 2 +- .../exec/planner/rm/TestMemoryCalculator.java | 55 +++-- .../drill/exec/record/TestRecordIterator.java | 4 +- .../exec/store/dfs/TestDrillFileSystem.java | 2 +- .../apache/drill/test/OperatorFixture.java | 2 +- .../drill/exec/proto/SchemaUserBitShared.java | 7 + .../drill/exec/proto/UserBitShared.java | 197 +++++++++++---- .../exec/proto/beans/OperatorProfile.java | 22 ++ .../src/main/protobuf/UserBitShared.proto | 1 + 29 files changed, 756 insertions(+), 353 deletions(-) diff --git a/contrib/native/client/src/protobuf/BitControl.pb.cc b/contrib/native/client/src/protobuf/BitControl.pb.cc index 3bf9db5b560..e31c7265a31 100644 --- a/contrib/native/client/src/protobuf/BitControl.pb.cc +++ b/contrib/native/client/src/protobuf/BitControl.pb.cc @@ -326,21 +326,23 @@ const ::google::protobuf::uint32 TableStruct::offsets[] GOOGLE_PROTOBUF_ATTRIBUT GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(::exec::bit::control::PlanFragment, options_json_), GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(::exec::bit::control::PlanFragment, context_), GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(::exec::bit::control::PlanFragment, collector_), - 2, - 7, + GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(::exec::bit::control::PlanFragment, endpointuuid_), + 3, 8, 9, 10, - 0, 11, - 3, - 4, + 0, 12, - 13, + 4, 5, - 1, + 13, + 14, 6, + 1, + 7, ~0u, + 2, GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(::exec::bit::control::Collector, _has_bits_), GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(::exec::bit::control::Collector, _internal_metadata_), ~0u, // no _extensions_ @@ -394,11 +396,11 @@ static const ::google::protobuf::internal::MigrationSchema schemas[] GOOGLE_PROT { 20, 27, sizeof(::exec::bit::control::FragmentStatus)}, { 29, 35, sizeof(::exec::bit::control::InitializeFragments)}, { 36, 43, sizeof(::exec::bit::control::CustomMessage)}, - { 45, 65, sizeof(::exec::bit::control::PlanFragment)}, - { 80, 89, sizeof(::exec::bit::control::Collector)}, - { 93, 102, sizeof(::exec::bit::control::QueryContextInformation)}, - { 106, 114, sizeof(::exec::bit::control::WorkQueueStatus)}, - { 117, 124, sizeof(::exec::bit::control::FinishedReceiver)}, + { 45, 66, sizeof(::exec::bit::control::PlanFragment)}, + { 82, 91, sizeof(::exec::bit::control::Collector)}, + { 95, 104, sizeof(::exec::bit::control::QueryContextInformation)}, + { 108, 116, sizeof(::exec::bit::control::WorkQueueStatus)}, + { 119, 126, sizeof(::exec::bit::control::FinishedReceiver)}, }; static ::google::protobuf::Message const * const file_default_instances[] = { @@ -449,7 +451,7 @@ void AddDescriptorsImpl() { "c.bit.FragmentHandle\"G\n\023InitializeFragme" "nts\0220\n\010fragment\030\001 \003(\0132\036.exec.bit.control" ".PlanFragment\".\n\rCustomMessage\022\014\n\004type\030\001" - " \001(\005\022\017\n\007message\030\002 \001(\014\"\374\003\n\014PlanFragment\022(" + " \001(\005\022\017\n\007message\030\002 \001(\014\"\222\004\n\014PlanFragment\022(" "\n\006handle\030\001 \001(\0132\030.exec.bit.FragmentHandle" "\022\024\n\014network_cost\030\004 \001(\002\022\020\n\010cpu_cost\030\005 \001(\002" "\022\021\n\tdisk_cost\030\006 \001(\002\022\023\n\013memory_cost\030\007 \001(\002" @@ -462,32 +464,33 @@ void AddDescriptorsImpl() { "ls\022\024\n\014options_json\030\017 \001(\t\022:\n\007context\030\020 \001(" "\0132).exec.bit.control.QueryContextInforma" "tion\022.\n\tcollector\030\021 \003(\0132\033.exec.bit.contr" - "ol.Collector\"\210\001\n\tCollector\022\"\n\032opposite_m" - "ajor_fragment_id\030\001 \001(\005\022#\n\027incoming_minor" - "_fragment\030\002 \003(\005B\002\020\001\022\035\n\025supports_out_of_o" - "rder\030\003 \001(\010\022\023\n\013is_spooling\030\004 \001(\010\"w\n\027Query" - "ContextInformation\022\030\n\020query_start_time\030\001" - " \001(\003\022\021\n\ttime_zone\030\002 \001(\005\022\033\n\023default_schem" - "a_name\030\003 \001(\t\022\022\n\nsession_id\030\004 \001(\t\"f\n\017Work" - "QueueStatus\022(\n\010endpoint\030\001 \001(\0132\026.exec.Dri" - "llbitEndpoint\022\024\n\014queue_length\030\002 \001(\005\022\023\n\013r" - "eport_time\030\003 \001(\003\"h\n\020FinishedReceiver\022*\n\010" - "receiver\030\001 \001(\0132\030.exec.bit.FragmentHandle" - "\022(\n\006sender\030\002 \001(\0132\030.exec.bit.FragmentHand" - "le*\206\003\n\007RpcType\022\r\n\tHANDSHAKE\020\000\022\007\n\003ACK\020\001\022\013" - "\n\007GOODBYE\020\002\022\034\n\030REQ_INITIALIZE_FRAGMENTS\020" - "\003\022\027\n\023REQ_CANCEL_FRAGMENT\020\006\022\031\n\025REQ_RECEIV" - "ER_FINISHED\020\007\022\027\n\023REQ_FRAGMENT_STATUS\020\010\022\022" - "\n\016REQ_BIT_STATUS\020\t\022\024\n\020REQ_QUERY_STATUS\020\n" - "\022\024\n\020REQ_QUERY_CANCEL\020\017\022\030\n\024REQ_UNPAUSE_FR" - "AGMENT\020\020\022\016\n\nREQ_CUSTOM\020\021\022\030\n\024RESP_FRAGMEN" - "T_HANDLE\020\013\022\030\n\024RESP_FRAGMENT_STATUS\020\014\022\023\n\017" - "RESP_BIT_STATUS\020\r\022\025\n\021RESP_QUERY_STATUS\020\016" - "\022\017\n\013RESP_CUSTOM\020\022\022\020\n\014SASL_MESSAGE\020\023B+\n\033o" - "rg.apache.drill.exec.protoB\nBitControlH\001" + "ol.Collector\022\024\n\014endpointUUID\030\022 \001(\t\"\210\001\n\tC" + "ollector\022\"\n\032opposite_major_fragment_id\030\001" + " \001(\005\022#\n\027incoming_minor_fragment\030\002 \003(\005B\002\020" + "\001\022\035\n\025supports_out_of_order\030\003 \001(\010\022\023\n\013is_s" + "pooling\030\004 \001(\010\"w\n\027QueryContextInformation" + "\022\030\n\020query_start_time\030\001 \001(\003\022\021\n\ttime_zone\030" + "\002 \001(\005\022\033\n\023default_schema_name\030\003 \001(\t\022\022\n\nse" + "ssion_id\030\004 \001(\t\"f\n\017WorkQueueStatus\022(\n\010end" + "point\030\001 \001(\0132\026.exec.DrillbitEndpoint\022\024\n\014q" + "ueue_length\030\002 \001(\005\022\023\n\013report_time\030\003 \001(\003\"h" + "\n\020FinishedReceiver\022*\n\010receiver\030\001 \001(\0132\030.e" + "xec.bit.FragmentHandle\022(\n\006sender\030\002 \001(\0132\030" + ".exec.bit.FragmentHandle*\206\003\n\007RpcType\022\r\n\t" + "HANDSHAKE\020\000\022\007\n\003ACK\020\001\022\013\n\007GOODBYE\020\002\022\034\n\030REQ" + "_INITIALIZE_FRAGMENTS\020\003\022\027\n\023REQ_CANCEL_FR" + "AGMENT\020\006\022\031\n\025REQ_RECEIVER_FINISHED\020\007\022\027\n\023R" + "EQ_FRAGMENT_STATUS\020\010\022\022\n\016REQ_BIT_STATUS\020\t" + "\022\024\n\020REQ_QUERY_STATUS\020\n\022\024\n\020REQ_QUERY_CANC" + "EL\020\017\022\030\n\024REQ_UNPAUSE_FRAGMENT\020\020\022\016\n\nREQ_CU" + "STOM\020\021\022\030\n\024RESP_FRAGMENT_HANDLE\020\013\022\030\n\024RESP" + "_FRAGMENT_STATUS\020\014\022\023\n\017RESP_BIT_STATUS\020\r\022" + "\025\n\021RESP_QUERY_STATUS\020\016\022\017\n\013RESP_CUSTOM\020\022\022" + "\020\n\014SASL_MESSAGE\020\023B+\n\033org.apache.drill.ex" + "ec.protoB\nBitControlH\001" }; ::google::protobuf::DescriptorPool::InternalAddGeneratedFile( - descriptor, 2000); + descriptor, 2022); ::google::protobuf::MessageFactory::InternalRegisterGeneratedFile( "BitControl.proto", &protobuf_RegisterTypes); ::protobuf_ExecutionProtos_2eproto::AddDescriptors(); @@ -2028,6 +2031,7 @@ const int PlanFragment::kCredentialsFieldNumber; const int PlanFragment::kOptionsJsonFieldNumber; const int PlanFragment::kContextFieldNumber; const int PlanFragment::kCollectorFieldNumber; +const int PlanFragment::kEndpointUUIDFieldNumber; #endif // !defined(_MSC_VER) || _MSC_VER >= 1900 PlanFragment::PlanFragment() @@ -2051,6 +2055,10 @@ PlanFragment::PlanFragment(const PlanFragment& from) if (from.has_options_json()) { options_json_.AssignWithDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), from.options_json_); } + endpointuuid_.UnsafeSetDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); + if (from.has_endpointuuid()) { + endpointuuid_.AssignWithDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), from.endpointuuid_); + } if (from.has_handle()) { handle_ = new ::exec::bit::FragmentHandle(*from.handle_); } else { @@ -2085,6 +2093,7 @@ PlanFragment::PlanFragment(const PlanFragment& from) void PlanFragment::SharedCtor() { fragment_json_.UnsafeSetDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); options_json_.UnsafeSetDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); + endpointuuid_.UnsafeSetDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); ::memset(&handle_, 0, static_cast( reinterpret_cast(&leaf_fragment_) - reinterpret_cast(&handle_)) + sizeof(leaf_fragment_)); @@ -2100,6 +2109,7 @@ PlanFragment::~PlanFragment() { void PlanFragment::SharedDtor() { fragment_json_.DestroyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); options_json_.DestroyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); + endpointuuid_.DestroyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); if (this != internal_default_instance()) delete handle_; if (this != internal_default_instance()) delete assignment_; if (this != internal_default_instance()) delete foreman_; @@ -2129,7 +2139,7 @@ void PlanFragment::Clear() { collector_.Clear(); cached_has_bits = _has_bits_[0]; - if (cached_has_bits & 127u) { + if (cached_has_bits & 255u) { if (cached_has_bits & 0x00000001u) { fragment_json_.ClearNonDefaultToEmptyNoArena(); } @@ -2137,31 +2147,33 @@ void PlanFragment::Clear() { options_json_.ClearNonDefaultToEmptyNoArena(); } if (cached_has_bits & 0x00000004u) { + endpointuuid_.ClearNonDefaultToEmptyNoArena(); + } + if (cached_has_bits & 0x00000008u) { GOOGLE_DCHECK(handle_ != NULL); handle_->Clear(); } - if (cached_has_bits & 0x00000008u) { + if (cached_has_bits & 0x00000010u) { GOOGLE_DCHECK(assignment_ != NULL); assignment_->Clear(); } - if (cached_has_bits & 0x00000010u) { + if (cached_has_bits & 0x00000020u) { GOOGLE_DCHECK(foreman_ != NULL); foreman_->Clear(); } - if (cached_has_bits & 0x00000020u) { + if (cached_has_bits & 0x00000040u) { GOOGLE_DCHECK(credentials_ != NULL); credentials_->Clear(); } - if (cached_has_bits & 0x00000040u) { + if (cached_has_bits & 0x00000080u) { GOOGLE_DCHECK(context_ != NULL); context_->Clear(); } } - network_cost_ = 0; - if (cached_has_bits & 16128u) { - ::memset(&cpu_cost_, 0, static_cast( + if (cached_has_bits & 32512u) { + ::memset(&network_cost_, 0, static_cast( reinterpret_cast(&leaf_fragment_) - - reinterpret_cast(&cpu_cost_)) + sizeof(leaf_fragment_)); + reinterpret_cast(&network_cost_)) + sizeof(leaf_fragment_)); mem_initial_ = GOOGLE_LONGLONG(20000000); mem_max_ = GOOGLE_LONGLONG(2000000000); } @@ -2381,6 +2393,22 @@ bool PlanFragment::MergePartialFromCodedStream( break; } + // optional string endpointUUID = 18; + case 18: { + if (static_cast< ::google::protobuf::uint8>(tag) == + static_cast< ::google::protobuf::uint8>(146u /* 146 & 0xFF */)) { + DO_(::google::protobuf::internal::WireFormatLite::ReadString( + input, this->mutable_endpointuuid())); + ::google::protobuf::internal::WireFormat::VerifyUTF8StringNamedField( + this->endpointuuid().data(), static_cast(this->endpointuuid().length()), + ::google::protobuf::internal::WireFormat::PARSE, + "exec.bit.control.PlanFragment.endpointUUID"); + } else { + goto handle_unusual; + } + break; + } + default: { handle_unusual: if (tag == 0) { @@ -2409,28 +2437,28 @@ void PlanFragment::SerializeWithCachedSizes( cached_has_bits = _has_bits_[0]; // optional .exec.bit.FragmentHandle handle = 1; - if (cached_has_bits & 0x00000004u) { + if (cached_has_bits & 0x00000008u) { ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( 1, this->_internal_handle(), output); } // optional float network_cost = 4; - if (cached_has_bits & 0x00000080u) { + if (cached_has_bits & 0x00000100u) { ::google::protobuf::internal::WireFormatLite::WriteFloat(4, this->network_cost(), output); } // optional float cpu_cost = 5; - if (cached_has_bits & 0x00000100u) { + if (cached_has_bits & 0x00000200u) { ::google::protobuf::internal::WireFormatLite::WriteFloat(5, this->cpu_cost(), output); } // optional float disk_cost = 6; - if (cached_has_bits & 0x00000200u) { + if (cached_has_bits & 0x00000400u) { ::google::protobuf::internal::WireFormatLite::WriteFloat(6, this->disk_cost(), output); } // optional float memory_cost = 7; - if (cached_has_bits & 0x00000400u) { + if (cached_has_bits & 0x00000800u) { ::google::protobuf::internal::WireFormatLite::WriteFloat(7, this->memory_cost(), output); } @@ -2445,34 +2473,34 @@ void PlanFragment::SerializeWithCachedSizes( } // optional bool leaf_fragment = 9; - if (cached_has_bits & 0x00000800u) { + if (cached_has_bits & 0x00001000u) { ::google::protobuf::internal::WireFormatLite::WriteBool(9, this->leaf_fragment(), output); } // optional .exec.DrillbitEndpoint assignment = 10; - if (cached_has_bits & 0x00000008u) { + if (cached_has_bits & 0x00000010u) { ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( 10, this->_internal_assignment(), output); } // optional .exec.DrillbitEndpoint foreman = 11; - if (cached_has_bits & 0x00000010u) { + if (cached_has_bits & 0x00000020u) { ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( 11, this->_internal_foreman(), output); } // optional int64 mem_initial = 12 [default = 20000000]; - if (cached_has_bits & 0x00001000u) { + if (cached_has_bits & 0x00002000u) { ::google::protobuf::internal::WireFormatLite::WriteInt64(12, this->mem_initial(), output); } // optional int64 mem_max = 13 [default = 2000000000]; - if (cached_has_bits & 0x00002000u) { + if (cached_has_bits & 0x00004000u) { ::google::protobuf::internal::WireFormatLite::WriteInt64(13, this->mem_max(), output); } // optional .exec.shared.UserCredentials credentials = 14; - if (cached_has_bits & 0x00000020u) { + if (cached_has_bits & 0x00000040u) { ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( 14, this->_internal_credentials(), output); } @@ -2488,7 +2516,7 @@ void PlanFragment::SerializeWithCachedSizes( } // optional .exec.bit.control.QueryContextInformation context = 16; - if (cached_has_bits & 0x00000040u) { + if (cached_has_bits & 0x00000080u) { ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( 16, this->_internal_context(), output); } @@ -2502,6 +2530,16 @@ void PlanFragment::SerializeWithCachedSizes( output); } + // optional string endpointUUID = 18; + if (cached_has_bits & 0x00000004u) { + ::google::protobuf::internal::WireFormat::VerifyUTF8StringNamedField( + this->endpointuuid().data(), static_cast(this->endpointuuid().length()), + ::google::protobuf::internal::WireFormat::SERIALIZE, + "exec.bit.control.PlanFragment.endpointUUID"); + ::google::protobuf::internal::WireFormatLite::WriteStringMaybeAliased( + 18, this->endpointuuid(), output); + } + if (_internal_metadata_.have_unknown_fields()) { ::google::protobuf::internal::WireFormat::SerializeUnknownFields( _internal_metadata_.unknown_fields(), output); @@ -2518,29 +2556,29 @@ ::google::protobuf::uint8* PlanFragment::InternalSerializeWithCachedSizesToArray cached_has_bits = _has_bits_[0]; // optional .exec.bit.FragmentHandle handle = 1; - if (cached_has_bits & 0x00000004u) { + if (cached_has_bits & 0x00000008u) { target = ::google::protobuf::internal::WireFormatLite:: InternalWriteMessageToArray( 1, this->_internal_handle(), deterministic, target); } // optional float network_cost = 4; - if (cached_has_bits & 0x00000080u) { + if (cached_has_bits & 0x00000100u) { target = ::google::protobuf::internal::WireFormatLite::WriteFloatToArray(4, this->network_cost(), target); } // optional float cpu_cost = 5; - if (cached_has_bits & 0x00000100u) { + if (cached_has_bits & 0x00000200u) { target = ::google::protobuf::internal::WireFormatLite::WriteFloatToArray(5, this->cpu_cost(), target); } // optional float disk_cost = 6; - if (cached_has_bits & 0x00000200u) { + if (cached_has_bits & 0x00000400u) { target = ::google::protobuf::internal::WireFormatLite::WriteFloatToArray(6, this->disk_cost(), target); } // optional float memory_cost = 7; - if (cached_has_bits & 0x00000400u) { + if (cached_has_bits & 0x00000800u) { target = ::google::protobuf::internal::WireFormatLite::WriteFloatToArray(7, this->memory_cost(), target); } @@ -2556,36 +2594,36 @@ ::google::protobuf::uint8* PlanFragment::InternalSerializeWithCachedSizesToArray } // optional bool leaf_fragment = 9; - if (cached_has_bits & 0x00000800u) { + if (cached_has_bits & 0x00001000u) { target = ::google::protobuf::internal::WireFormatLite::WriteBoolToArray(9, this->leaf_fragment(), target); } // optional .exec.DrillbitEndpoint assignment = 10; - if (cached_has_bits & 0x00000008u) { + if (cached_has_bits & 0x00000010u) { target = ::google::protobuf::internal::WireFormatLite:: InternalWriteMessageToArray( 10, this->_internal_assignment(), deterministic, target); } // optional .exec.DrillbitEndpoint foreman = 11; - if (cached_has_bits & 0x00000010u) { + if (cached_has_bits & 0x00000020u) { target = ::google::protobuf::internal::WireFormatLite:: InternalWriteMessageToArray( 11, this->_internal_foreman(), deterministic, target); } // optional int64 mem_initial = 12 [default = 20000000]; - if (cached_has_bits & 0x00001000u) { + if (cached_has_bits & 0x00002000u) { target = ::google::protobuf::internal::WireFormatLite::WriteInt64ToArray(12, this->mem_initial(), target); } // optional int64 mem_max = 13 [default = 2000000000]; - if (cached_has_bits & 0x00002000u) { + if (cached_has_bits & 0x00004000u) { target = ::google::protobuf::internal::WireFormatLite::WriteInt64ToArray(13, this->mem_max(), target); } // optional .exec.shared.UserCredentials credentials = 14; - if (cached_has_bits & 0x00000020u) { + if (cached_has_bits & 0x00000040u) { target = ::google::protobuf::internal::WireFormatLite:: InternalWriteMessageToArray( 14, this->_internal_credentials(), deterministic, target); @@ -2603,7 +2641,7 @@ ::google::protobuf::uint8* PlanFragment::InternalSerializeWithCachedSizesToArray } // optional .exec.bit.control.QueryContextInformation context = 16; - if (cached_has_bits & 0x00000040u) { + if (cached_has_bits & 0x00000080u) { target = ::google::protobuf::internal::WireFormatLite:: InternalWriteMessageToArray( 16, this->_internal_context(), deterministic, target); @@ -2617,6 +2655,17 @@ ::google::protobuf::uint8* PlanFragment::InternalSerializeWithCachedSizesToArray 17, this->collector(static_cast(i)), deterministic, target); } + // optional string endpointUUID = 18; + if (cached_has_bits & 0x00000004u) { + ::google::protobuf::internal::WireFormat::VerifyUTF8StringNamedField( + this->endpointuuid().data(), static_cast(this->endpointuuid().length()), + ::google::protobuf::internal::WireFormat::SERIALIZE, + "exec.bit.control.PlanFragment.endpointUUID"); + target = + ::google::protobuf::internal::WireFormatLite::WriteStringToArray( + 18, this->endpointuuid(), target); + } + if (_internal_metadata_.have_unknown_fields()) { target = ::google::protobuf::internal::WireFormat::SerializeUnknownFieldsToArray( _internal_metadata_.unknown_fields(), target); @@ -2660,6 +2709,13 @@ size_t PlanFragment::ByteSizeLong() const { this->options_json()); } + // optional string endpointUUID = 18; + if (has_endpointuuid()) { + total_size += 2 + + ::google::protobuf::internal::WireFormatLite::StringSize( + this->endpointuuid()); + } + // optional .exec.bit.FragmentHandle handle = 1; if (has_handle()) { total_size += 1 + @@ -2695,13 +2751,13 @@ size_t PlanFragment::ByteSizeLong() const { *context_); } + } + if (_has_bits_[8 / 32] & 32512u) { // optional float network_cost = 4; if (has_network_cost()) { total_size += 1 + 4; } - } - if (_has_bits_[8 / 32] & 16128u) { // optional float cpu_cost = 5; if (has_cpu_cost()) { total_size += 1 + 4; @@ -2776,42 +2832,45 @@ void PlanFragment::MergeFrom(const PlanFragment& from) { options_json_.AssignWithDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), from.options_json_); } if (cached_has_bits & 0x00000004u) { - mutable_handle()->::exec::bit::FragmentHandle::MergeFrom(from.handle()); + set_has_endpointuuid(); + endpointuuid_.AssignWithDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), from.endpointuuid_); } if (cached_has_bits & 0x00000008u) { - mutable_assignment()->::exec::DrillbitEndpoint::MergeFrom(from.assignment()); + mutable_handle()->::exec::bit::FragmentHandle::MergeFrom(from.handle()); } if (cached_has_bits & 0x00000010u) { - mutable_foreman()->::exec::DrillbitEndpoint::MergeFrom(from.foreman()); + mutable_assignment()->::exec::DrillbitEndpoint::MergeFrom(from.assignment()); } if (cached_has_bits & 0x00000020u) { - mutable_credentials()->::exec::shared::UserCredentials::MergeFrom(from.credentials()); + mutable_foreman()->::exec::DrillbitEndpoint::MergeFrom(from.foreman()); } if (cached_has_bits & 0x00000040u) { - mutable_context()->::exec::bit::control::QueryContextInformation::MergeFrom(from.context()); + mutable_credentials()->::exec::shared::UserCredentials::MergeFrom(from.credentials()); } if (cached_has_bits & 0x00000080u) { - network_cost_ = from.network_cost_; + mutable_context()->::exec::bit::control::QueryContextInformation::MergeFrom(from.context()); } - _has_bits_[0] |= cached_has_bits; } - if (cached_has_bits & 16128u) { + if (cached_has_bits & 32512u) { if (cached_has_bits & 0x00000100u) { - cpu_cost_ = from.cpu_cost_; + network_cost_ = from.network_cost_; } if (cached_has_bits & 0x00000200u) { - disk_cost_ = from.disk_cost_; + cpu_cost_ = from.cpu_cost_; } if (cached_has_bits & 0x00000400u) { - memory_cost_ = from.memory_cost_; + disk_cost_ = from.disk_cost_; } if (cached_has_bits & 0x00000800u) { - leaf_fragment_ = from.leaf_fragment_; + memory_cost_ = from.memory_cost_; } if (cached_has_bits & 0x00001000u) { - mem_initial_ = from.mem_initial_; + leaf_fragment_ = from.leaf_fragment_; } if (cached_has_bits & 0x00002000u) { + mem_initial_ = from.mem_initial_; + } + if (cached_has_bits & 0x00004000u) { mem_max_ = from.mem_max_; } _has_bits_[0] |= cached_has_bits; @@ -2847,6 +2906,8 @@ void PlanFragment::InternalSwap(PlanFragment* other) { GetArenaNoVirtual()); options_json_.Swap(&other->options_json_, &::google::protobuf::internal::GetEmptyStringAlreadyInited(), GetArenaNoVirtual()); + endpointuuid_.Swap(&other->endpointuuid_, &::google::protobuf::internal::GetEmptyStringAlreadyInited(), + GetArenaNoVirtual()); swap(handle_, other->handle_); swap(assignment_, other->assignment_); swap(foreman_, other->foreman_); diff --git a/contrib/native/client/src/protobuf/BitControl.pb.h b/contrib/native/client/src/protobuf/BitControl.pb.h index abfda10d619..58e4889946f 100644 --- a/contrib/native/client/src/protobuf/BitControl.pb.h +++ b/contrib/native/client/src/protobuf/BitControl.pb.h @@ -938,6 +938,21 @@ class PlanFragment : public ::google::protobuf::Message /* @@protoc_insertion_po ::std::string* release_options_json(); void set_allocated_options_json(::std::string* options_json); + // optional string endpointUUID = 18; + bool has_endpointuuid() const; + void clear_endpointuuid(); + static const int kEndpointUUIDFieldNumber = 18; + const ::std::string& endpointuuid() const; + void set_endpointuuid(const ::std::string& value); + #if LANG_CXX11 + void set_endpointuuid(::std::string&& value); + #endif + void set_endpointuuid(const char* value); + void set_endpointuuid(const char* value, size_t size); + ::std::string* mutable_endpointuuid(); + ::std::string* release_endpointuuid(); + void set_allocated_endpointuuid(::std::string* endpointuuid); + // optional .exec.bit.FragmentHandle handle = 1; bool has_handle() const; void clear_handle(); @@ -1077,6 +1092,8 @@ class PlanFragment : public ::google::protobuf::Message /* @@protoc_insertion_po void clear_has_options_json(); void set_has_context(); void clear_has_context(); + void set_has_endpointuuid(); + void clear_has_endpointuuid(); ::google::protobuf::internal::InternalMetadataWithArena _internal_metadata_; ::google::protobuf::internal::HasBits<1> _has_bits_; @@ -1084,6 +1101,7 @@ class PlanFragment : public ::google::protobuf::Message /* @@protoc_insertion_po ::google::protobuf::RepeatedPtrField< ::exec::bit::control::Collector > collector_; ::google::protobuf::internal::ArenaStringPtr fragment_json_; ::google::protobuf::internal::ArenaStringPtr options_json_; + ::google::protobuf::internal::ArenaStringPtr endpointuuid_; ::exec::bit::FragmentHandle* handle_; ::exec::DrillbitEndpoint* assignment_; ::exec::DrillbitEndpoint* foreman_; @@ -2142,13 +2160,13 @@ inline void CustomMessage::set_allocated_message(::std::string* message) { // optional .exec.bit.FragmentHandle handle = 1; inline bool PlanFragment::has_handle() const { - return (_has_bits_[0] & 0x00000004u) != 0; + return (_has_bits_[0] & 0x00000008u) != 0; } inline void PlanFragment::set_has_handle() { - _has_bits_[0] |= 0x00000004u; + _has_bits_[0] |= 0x00000008u; } inline void PlanFragment::clear_has_handle() { - _has_bits_[0] &= ~0x00000004u; + _has_bits_[0] &= ~0x00000008u; } inline const ::exec::bit::FragmentHandle& PlanFragment::_internal_handle() const { return *handle_; @@ -2196,13 +2214,13 @@ inline void PlanFragment::set_allocated_handle(::exec::bit::FragmentHandle* hand // optional float network_cost = 4; inline bool PlanFragment::has_network_cost() const { - return (_has_bits_[0] & 0x00000080u) != 0; + return (_has_bits_[0] & 0x00000100u) != 0; } inline void PlanFragment::set_has_network_cost() { - _has_bits_[0] |= 0x00000080u; + _has_bits_[0] |= 0x00000100u; } inline void PlanFragment::clear_has_network_cost() { - _has_bits_[0] &= ~0x00000080u; + _has_bits_[0] &= ~0x00000100u; } inline void PlanFragment::clear_network_cost() { network_cost_ = 0; @@ -2220,13 +2238,13 @@ inline void PlanFragment::set_network_cost(float value) { // optional float cpu_cost = 5; inline bool PlanFragment::has_cpu_cost() const { - return (_has_bits_[0] & 0x00000100u) != 0; + return (_has_bits_[0] & 0x00000200u) != 0; } inline void PlanFragment::set_has_cpu_cost() { - _has_bits_[0] |= 0x00000100u; + _has_bits_[0] |= 0x00000200u; } inline void PlanFragment::clear_has_cpu_cost() { - _has_bits_[0] &= ~0x00000100u; + _has_bits_[0] &= ~0x00000200u; } inline void PlanFragment::clear_cpu_cost() { cpu_cost_ = 0; @@ -2244,13 +2262,13 @@ inline void PlanFragment::set_cpu_cost(float value) { // optional float disk_cost = 6; inline bool PlanFragment::has_disk_cost() const { - return (_has_bits_[0] & 0x00000200u) != 0; + return (_has_bits_[0] & 0x00000400u) != 0; } inline void PlanFragment::set_has_disk_cost() { - _has_bits_[0] |= 0x00000200u; + _has_bits_[0] |= 0x00000400u; } inline void PlanFragment::clear_has_disk_cost() { - _has_bits_[0] &= ~0x00000200u; + _has_bits_[0] &= ~0x00000400u; } inline void PlanFragment::clear_disk_cost() { disk_cost_ = 0; @@ -2268,13 +2286,13 @@ inline void PlanFragment::set_disk_cost(float value) { // optional float memory_cost = 7; inline bool PlanFragment::has_memory_cost() const { - return (_has_bits_[0] & 0x00000400u) != 0; + return (_has_bits_[0] & 0x00000800u) != 0; } inline void PlanFragment::set_has_memory_cost() { - _has_bits_[0] |= 0x00000400u; + _has_bits_[0] |= 0x00000800u; } inline void PlanFragment::clear_has_memory_cost() { - _has_bits_[0] &= ~0x00000400u; + _has_bits_[0] &= ~0x00000800u; } inline void PlanFragment::clear_memory_cost() { memory_cost_ = 0; @@ -2358,13 +2376,13 @@ inline void PlanFragment::set_allocated_fragment_json(::std::string* fragment_js // optional bool leaf_fragment = 9; inline bool PlanFragment::has_leaf_fragment() const { - return (_has_bits_[0] & 0x00000800u) != 0; + return (_has_bits_[0] & 0x00001000u) != 0; } inline void PlanFragment::set_has_leaf_fragment() { - _has_bits_[0] |= 0x00000800u; + _has_bits_[0] |= 0x00001000u; } inline void PlanFragment::clear_has_leaf_fragment() { - _has_bits_[0] &= ~0x00000800u; + _has_bits_[0] &= ~0x00001000u; } inline void PlanFragment::clear_leaf_fragment() { leaf_fragment_ = false; @@ -2382,13 +2400,13 @@ inline void PlanFragment::set_leaf_fragment(bool value) { // optional .exec.DrillbitEndpoint assignment = 10; inline bool PlanFragment::has_assignment() const { - return (_has_bits_[0] & 0x00000008u) != 0; + return (_has_bits_[0] & 0x00000010u) != 0; } inline void PlanFragment::set_has_assignment() { - _has_bits_[0] |= 0x00000008u; + _has_bits_[0] |= 0x00000010u; } inline void PlanFragment::clear_has_assignment() { - _has_bits_[0] &= ~0x00000008u; + _has_bits_[0] &= ~0x00000010u; } inline const ::exec::DrillbitEndpoint& PlanFragment::_internal_assignment() const { return *assignment_; @@ -2436,13 +2454,13 @@ inline void PlanFragment::set_allocated_assignment(::exec::DrillbitEndpoint* ass // optional .exec.DrillbitEndpoint foreman = 11; inline bool PlanFragment::has_foreman() const { - return (_has_bits_[0] & 0x00000010u) != 0; + return (_has_bits_[0] & 0x00000020u) != 0; } inline void PlanFragment::set_has_foreman() { - _has_bits_[0] |= 0x00000010u; + _has_bits_[0] |= 0x00000020u; } inline void PlanFragment::clear_has_foreman() { - _has_bits_[0] &= ~0x00000010u; + _has_bits_[0] &= ~0x00000020u; } inline const ::exec::DrillbitEndpoint& PlanFragment::_internal_foreman() const { return *foreman_; @@ -2490,13 +2508,13 @@ inline void PlanFragment::set_allocated_foreman(::exec::DrillbitEndpoint* forema // optional int64 mem_initial = 12 [default = 20000000]; inline bool PlanFragment::has_mem_initial() const { - return (_has_bits_[0] & 0x00001000u) != 0; + return (_has_bits_[0] & 0x00002000u) != 0; } inline void PlanFragment::set_has_mem_initial() { - _has_bits_[0] |= 0x00001000u; + _has_bits_[0] |= 0x00002000u; } inline void PlanFragment::clear_has_mem_initial() { - _has_bits_[0] &= ~0x00001000u; + _has_bits_[0] &= ~0x00002000u; } inline void PlanFragment::clear_mem_initial() { mem_initial_ = GOOGLE_LONGLONG(20000000); @@ -2514,13 +2532,13 @@ inline void PlanFragment::set_mem_initial(::google::protobuf::int64 value) { // optional int64 mem_max = 13 [default = 2000000000]; inline bool PlanFragment::has_mem_max() const { - return (_has_bits_[0] & 0x00002000u) != 0; + return (_has_bits_[0] & 0x00004000u) != 0; } inline void PlanFragment::set_has_mem_max() { - _has_bits_[0] |= 0x00002000u; + _has_bits_[0] |= 0x00004000u; } inline void PlanFragment::clear_has_mem_max() { - _has_bits_[0] &= ~0x00002000u; + _has_bits_[0] &= ~0x00004000u; } inline void PlanFragment::clear_mem_max() { mem_max_ = GOOGLE_LONGLONG(2000000000); @@ -2538,13 +2556,13 @@ inline void PlanFragment::set_mem_max(::google::protobuf::int64 value) { // optional .exec.shared.UserCredentials credentials = 14; inline bool PlanFragment::has_credentials() const { - return (_has_bits_[0] & 0x00000020u) != 0; + return (_has_bits_[0] & 0x00000040u) != 0; } inline void PlanFragment::set_has_credentials() { - _has_bits_[0] |= 0x00000020u; + _has_bits_[0] |= 0x00000040u; } inline void PlanFragment::clear_has_credentials() { - _has_bits_[0] &= ~0x00000020u; + _has_bits_[0] &= ~0x00000040u; } inline const ::exec::shared::UserCredentials& PlanFragment::_internal_credentials() const { return *credentials_; @@ -2658,13 +2676,13 @@ inline void PlanFragment::set_allocated_options_json(::std::string* options_json // optional .exec.bit.control.QueryContextInformation context = 16; inline bool PlanFragment::has_context() const { - return (_has_bits_[0] & 0x00000040u) != 0; + return (_has_bits_[0] & 0x00000080u) != 0; } inline void PlanFragment::set_has_context() { - _has_bits_[0] |= 0x00000040u; + _has_bits_[0] |= 0x00000080u; } inline void PlanFragment::clear_has_context() { - _has_bits_[0] &= ~0x00000040u; + _has_bits_[0] &= ~0x00000080u; } inline void PlanFragment::clear_context() { if (context_ != NULL) context_->Clear(); @@ -2744,6 +2762,72 @@ PlanFragment::collector() const { return collector_; } +// optional string endpointUUID = 18; +inline bool PlanFragment::has_endpointuuid() const { + return (_has_bits_[0] & 0x00000004u) != 0; +} +inline void PlanFragment::set_has_endpointuuid() { + _has_bits_[0] |= 0x00000004u; +} +inline void PlanFragment::clear_has_endpointuuid() { + _has_bits_[0] &= ~0x00000004u; +} +inline void PlanFragment::clear_endpointuuid() { + endpointuuid_.ClearToEmptyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); + clear_has_endpointuuid(); +} +inline const ::std::string& PlanFragment::endpointuuid() const { + // @@protoc_insertion_point(field_get:exec.bit.control.PlanFragment.endpointUUID) + return endpointuuid_.GetNoArena(); +} +inline void PlanFragment::set_endpointuuid(const ::std::string& value) { + set_has_endpointuuid(); + endpointuuid_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), value); + // @@protoc_insertion_point(field_set:exec.bit.control.PlanFragment.endpointUUID) +} +#if LANG_CXX11 +inline void PlanFragment::set_endpointuuid(::std::string&& value) { + set_has_endpointuuid(); + endpointuuid_.SetNoArena( + &::google::protobuf::internal::GetEmptyStringAlreadyInited(), ::std::move(value)); + // @@protoc_insertion_point(field_set_rvalue:exec.bit.control.PlanFragment.endpointUUID) +} +#endif +inline void PlanFragment::set_endpointuuid(const char* value) { + GOOGLE_DCHECK(value != NULL); + set_has_endpointuuid(); + endpointuuid_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), ::std::string(value)); + // @@protoc_insertion_point(field_set_char:exec.bit.control.PlanFragment.endpointUUID) +} +inline void PlanFragment::set_endpointuuid(const char* value, size_t size) { + set_has_endpointuuid(); + endpointuuid_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), + ::std::string(reinterpret_cast(value), size)); + // @@protoc_insertion_point(field_set_pointer:exec.bit.control.PlanFragment.endpointUUID) +} +inline ::std::string* PlanFragment::mutable_endpointuuid() { + set_has_endpointuuid(); + // @@protoc_insertion_point(field_mutable:exec.bit.control.PlanFragment.endpointUUID) + return endpointuuid_.MutableNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); +} +inline ::std::string* PlanFragment::release_endpointuuid() { + // @@protoc_insertion_point(field_release:exec.bit.control.PlanFragment.endpointUUID) + if (!has_endpointuuid()) { + return NULL; + } + clear_has_endpointuuid(); + return endpointuuid_.ReleaseNonDefaultNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); +} +inline void PlanFragment::set_allocated_endpointuuid(::std::string* endpointuuid) { + if (endpointuuid != NULL) { + set_has_endpointuuid(); + } else { + clear_has_endpointuuid(); + } + endpointuuid_.SetAllocatedNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), endpointuuid); + // @@protoc_insertion_point(field_set_allocated:exec.bit.control.PlanFragment.endpointUUID) +} + // ------------------------------------------------------------------- // Collector diff --git a/contrib/native/client/src/protobuf/UserBitShared.pb.cc b/contrib/native/client/src/protobuf/UserBitShared.pb.cc index 0996fca902a..80e728a8834 100644 --- a/contrib/native/client/src/protobuf/UserBitShared.pb.cc +++ b/contrib/native/client/src/protobuf/UserBitShared.pb.cc @@ -800,6 +800,7 @@ const ::google::protobuf::uint32 TableStruct::offsets[] GOOGLE_PROTOBUF_ATTRIBUT GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(::exec::shared::OperatorProfile, peak_local_memory_allocated_), GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(::exec::shared::OperatorProfile, metric_), GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(::exec::shared::OperatorProfile, wait_nanos_), + GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(::exec::shared::OperatorProfile, optimal_mem_allocation_), ~0u, 0, 1, @@ -808,6 +809,7 @@ const ::google::protobuf::uint32 TableStruct::offsets[] GOOGLE_PROTOBUF_ATTRIBUT 4, ~0u, 5, + 6, GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(::exec::shared::StreamProfile, _has_bits_), GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(::exec::shared::StreamProfile, _internal_metadata_), ~0u, // no _extensions_ @@ -875,12 +877,12 @@ static const ::google::protobuf::internal::MigrationSchema schemas[] GOOGLE_PROT { 169, 197, sizeof(::exec::shared::QueryProfile)}, { 220, 227, sizeof(::exec::shared::MajorFragmentProfile)}, { 229, 245, sizeof(::exec::shared::MinorFragmentProfile)}, - { 256, 269, sizeof(::exec::shared::OperatorProfile)}, - { 277, 285, sizeof(::exec::shared::StreamProfile)}, - { 288, 296, sizeof(::exec::shared::MetricValue)}, - { 299, 305, sizeof(::exec::shared::Registry)}, - { 306, 313, sizeof(::exec::shared::Jar)}, - { 315, 323, sizeof(::exec::shared::SaslMessage)}, + { 256, 270, sizeof(::exec::shared::OperatorProfile)}, + { 279, 287, sizeof(::exec::shared::StreamProfile)}, + { 290, 298, sizeof(::exec::shared::MetricValue)}, + { 301, 307, sizeof(::exec::shared::Registry)}, + { 308, 315, sizeof(::exec::shared::Jar)}, + { 317, 325, sizeof(::exec::shared::SaslMessage)}, }; static ::google::protobuf::Message const * const file_default_instances[] = { @@ -1013,67 +1015,68 @@ void AddDescriptorsImpl() { "y_used\030\007 \001(\003\022\027\n\017max_memory_used\030\010 \001(\003\022(\n" "\010endpoint\030\t \001(\0132\026.exec.DrillbitEndpoint\022" "\023\n\013last_update\030\n \001(\003\022\025\n\rlast_progress\030\013 " - "\001(\003\"\377\001\n\017OperatorProfile\0221\n\rinput_profile" + "\001(\003\"\237\002\n\017OperatorProfile\0221\n\rinput_profile" "\030\001 \003(\0132\032.exec.shared.StreamProfile\022\023\n\013op" "erator_id\030\003 \001(\005\022\025\n\roperator_type\030\004 \001(\005\022\023" "\n\013setup_nanos\030\005 \001(\003\022\025\n\rprocess_nanos\030\006 \001" "(\003\022#\n\033peak_local_memory_allocated\030\007 \001(\003\022" "(\n\006metric\030\010 \003(\0132\030.exec.shared.MetricValu" - "e\022\022\n\nwait_nanos\030\t \001(\003\"B\n\rStreamProfile\022\017" - "\n\007records\030\001 \001(\003\022\017\n\007batches\030\002 \001(\003\022\017\n\007sche" - "mas\030\003 \001(\003\"J\n\013MetricValue\022\021\n\tmetric_id\030\001 " - "\001(\005\022\022\n\nlong_value\030\002 \001(\003\022\024\n\014double_value\030" - "\003 \001(\001\")\n\010Registry\022\035\n\003jar\030\001 \003(\0132\020.exec.sh" - "ared.Jar\"/\n\003Jar\022\014\n\004name\030\001 \001(\t\022\032\n\022functio" - "n_signature\030\002 \003(\t\"W\n\013SaslMessage\022\021\n\tmech" - "anism\030\001 \001(\t\022\014\n\004data\030\002 \001(\014\022\'\n\006status\030\003 \001(" - "\0162\027.exec.shared.SaslStatus*5\n\nRpcChannel" - "\022\017\n\013BIT_CONTROL\020\000\022\014\n\010BIT_DATA\020\001\022\010\n\004USER\020" - "\002*V\n\tQueryType\022\007\n\003SQL\020\001\022\013\n\007LOGICAL\020\002\022\014\n\010" - "PHYSICAL\020\003\022\r\n\tEXECUTION\020\004\022\026\n\022PREPARED_ST" - "ATEMENT\020\005*\207\001\n\rFragmentState\022\013\n\007SENDING\020\000" - "\022\027\n\023AWAITING_ALLOCATION\020\001\022\013\n\007RUNNING\020\002\022\014" - "\n\010FINISHED\020\003\022\r\n\tCANCELLED\020\004\022\n\n\006FAILED\020\005\022" - "\032\n\026CANCELLATION_REQUESTED\020\006*\374\t\n\020CoreOper" - "atorType\022\021\n\rSINGLE_SENDER\020\000\022\024\n\020BROADCAST" - "_SENDER\020\001\022\n\n\006FILTER\020\002\022\022\n\016HASH_AGGREGATE\020" - "\003\022\r\n\tHASH_JOIN\020\004\022\016\n\nMERGE_JOIN\020\005\022\031\n\025HASH" - "_PARTITION_SENDER\020\006\022\t\n\005LIMIT\020\007\022\024\n\020MERGIN" - "G_RECEIVER\020\010\022\034\n\030ORDERED_PARTITION_SENDER" - "\020\t\022\013\n\007PROJECT\020\n\022\026\n\022UNORDERED_RECEIVER\020\013\022" - "\032\n\026RANGE_PARTITION_SENDER\020\014\022\n\n\006SCREEN\020\r\022" - "\034\n\030SELECTION_VECTOR_REMOVER\020\016\022\027\n\023STREAMI" - "NG_AGGREGATE\020\017\022\016\n\nTOP_N_SORT\020\020\022\021\n\rEXTERN" - "AL_SORT\020\021\022\t\n\005TRACE\020\022\022\t\n\005UNION\020\023\022\014\n\010OLD_S" - "ORT\020\024\022\032\n\026PARQUET_ROW_GROUP_SCAN\020\025\022\021\n\rHIV" - "E_SUB_SCAN\020\026\022\025\n\021SYSTEM_TABLE_SCAN\020\027\022\021\n\rM" - "OCK_SUB_SCAN\020\030\022\022\n\016PARQUET_WRITER\020\031\022\023\n\017DI" - "RECT_SUB_SCAN\020\032\022\017\n\013TEXT_WRITER\020\033\022\021\n\rTEXT" - "_SUB_SCAN\020\034\022\021\n\rJSON_SUB_SCAN\020\035\022\030\n\024INFO_S" - "CHEMA_SUB_SCAN\020\036\022\023\n\017COMPLEX_TO_JSON\020\037\022\025\n" - "\021PRODUCER_CONSUMER\020 \022\022\n\016HBASE_SUB_SCAN\020!" - "\022\n\n\006WINDOW\020\"\022\024\n\020NESTED_LOOP_JOIN\020#\022\021\n\rAV" - "RO_SUB_SCAN\020$\022\021\n\rPCAP_SUB_SCAN\020%\022\022\n\016KAFK" - "A_SUB_SCAN\020&\022\021\n\rKUDU_SUB_SCAN\020\'\022\013\n\007FLATT" - "EN\020(\022\020\n\014LATERAL_JOIN\020)\022\n\n\006UNNEST\020*\022,\n(HI" - "VE_DRILL_NATIVE_PARQUET_ROW_GROUP_SCAN\020+" - "\022\r\n\tJDBC_SCAN\020,\022\022\n\016REGEX_SUB_SCAN\020-\022\023\n\017M" - "APRDB_SUB_SCAN\020.\022\022\n\016MONGO_SUB_SCAN\020/\022\017\n\013" - "KUDU_WRITER\0200\022\026\n\022OPEN_TSDB_SUB_SCAN\0201\022\017\n" - "\013JSON_WRITER\0202\022\026\n\022HTPPD_LOG_SUB_SCAN\0203\022\022" - "\n\016IMAGE_SUB_SCAN\0204\022\025\n\021SEQUENCE_SUB_SCAN\020" - "5\022\023\n\017PARTITION_LIMIT\0206\022\023\n\017PCAPNG_SUB_SCA" - "N\0207\022\022\n\016RUNTIME_FILTER\0208\022\017\n\013ROWKEY_JOIN\0209" - "\022\023\n\017SYSLOG_SUB_SCAN\020:\022\030\n\024STATISTICS_AGGR" - "EGATE\020;\022\020\n\014UNPIVOT_MAPS\020<\022\024\n\020STATISTICS_" - "MERGE\020=\022\021\n\rLTSV_SUB_SCAN\020>*g\n\nSaslStatus" - "\022\020\n\014SASL_UNKNOWN\020\000\022\016\n\nSASL_START\020\001\022\024\n\020SA" - "SL_IN_PROGRESS\020\002\022\020\n\014SASL_SUCCESS\020\003\022\017\n\013SA" - "SL_FAILED\020\004B.\n\033org.apache.drill.exec.pro" - "toB\rUserBitSharedH\001" + "e\022\022\n\nwait_nanos\030\t \001(\003\022\036\n\026optimal_mem_all" + "ocation\030\n \001(\003\"B\n\rStreamProfile\022\017\n\007record" + "s\030\001 \001(\003\022\017\n\007batches\030\002 \001(\003\022\017\n\007schemas\030\003 \001(" + "\003\"J\n\013MetricValue\022\021\n\tmetric_id\030\001 \001(\005\022\022\n\nl" + "ong_value\030\002 \001(\003\022\024\n\014double_value\030\003 \001(\001\")\n" + "\010Registry\022\035\n\003jar\030\001 \003(\0132\020.exec.shared.Jar" + "\"/\n\003Jar\022\014\n\004name\030\001 \001(\t\022\032\n\022function_signat" + "ure\030\002 \003(\t\"W\n\013SaslMessage\022\021\n\tmechanism\030\001 " + "\001(\t\022\014\n\004data\030\002 \001(\014\022\'\n\006status\030\003 \001(\0162\027.exec" + ".shared.SaslStatus*5\n\nRpcChannel\022\017\n\013BIT_" + "CONTROL\020\000\022\014\n\010BIT_DATA\020\001\022\010\n\004USER\020\002*V\n\tQue" + "ryType\022\007\n\003SQL\020\001\022\013\n\007LOGICAL\020\002\022\014\n\010PHYSICAL" + "\020\003\022\r\n\tEXECUTION\020\004\022\026\n\022PREPARED_STATEMENT\020" + "\005*\207\001\n\rFragmentState\022\013\n\007SENDING\020\000\022\027\n\023AWAI" + "TING_ALLOCATION\020\001\022\013\n\007RUNNING\020\002\022\014\n\010FINISH" + "ED\020\003\022\r\n\tCANCELLED\020\004\022\n\n\006FAILED\020\005\022\032\n\026CANCE" + "LLATION_REQUESTED\020\006*\374\t\n\020CoreOperatorType" + "\022\021\n\rSINGLE_SENDER\020\000\022\024\n\020BROADCAST_SENDER\020" + "\001\022\n\n\006FILTER\020\002\022\022\n\016HASH_AGGREGATE\020\003\022\r\n\tHAS" + "H_JOIN\020\004\022\016\n\nMERGE_JOIN\020\005\022\031\n\025HASH_PARTITI" + "ON_SENDER\020\006\022\t\n\005LIMIT\020\007\022\024\n\020MERGING_RECEIV" + "ER\020\010\022\034\n\030ORDERED_PARTITION_SENDER\020\t\022\013\n\007PR" + "OJECT\020\n\022\026\n\022UNORDERED_RECEIVER\020\013\022\032\n\026RANGE" + "_PARTITION_SENDER\020\014\022\n\n\006SCREEN\020\r\022\034\n\030SELEC" + "TION_VECTOR_REMOVER\020\016\022\027\n\023STREAMING_AGGRE" + "GATE\020\017\022\016\n\nTOP_N_SORT\020\020\022\021\n\rEXTERNAL_SORT\020" + "\021\022\t\n\005TRACE\020\022\022\t\n\005UNION\020\023\022\014\n\010OLD_SORT\020\024\022\032\n" + "\026PARQUET_ROW_GROUP_SCAN\020\025\022\021\n\rHIVE_SUB_SC" + "AN\020\026\022\025\n\021SYSTEM_TABLE_SCAN\020\027\022\021\n\rMOCK_SUB_" + "SCAN\020\030\022\022\n\016PARQUET_WRITER\020\031\022\023\n\017DIRECT_SUB" + "_SCAN\020\032\022\017\n\013TEXT_WRITER\020\033\022\021\n\rTEXT_SUB_SCA" + "N\020\034\022\021\n\rJSON_SUB_SCAN\020\035\022\030\n\024INFO_SCHEMA_SU" + "B_SCAN\020\036\022\023\n\017COMPLEX_TO_JSON\020\037\022\025\n\021PRODUCE" + "R_CONSUMER\020 \022\022\n\016HBASE_SUB_SCAN\020!\022\n\n\006WIND" + "OW\020\"\022\024\n\020NESTED_LOOP_JOIN\020#\022\021\n\rAVRO_SUB_S" + "CAN\020$\022\021\n\rPCAP_SUB_SCAN\020%\022\022\n\016KAFKA_SUB_SC" + "AN\020&\022\021\n\rKUDU_SUB_SCAN\020\'\022\013\n\007FLATTEN\020(\022\020\n\014" + "LATERAL_JOIN\020)\022\n\n\006UNNEST\020*\022,\n(HIVE_DRILL" + "_NATIVE_PARQUET_ROW_GROUP_SCAN\020+\022\r\n\tJDBC" + "_SCAN\020,\022\022\n\016REGEX_SUB_SCAN\020-\022\023\n\017MAPRDB_SU" + "B_SCAN\020.\022\022\n\016MONGO_SUB_SCAN\020/\022\017\n\013KUDU_WRI" + "TER\0200\022\026\n\022OPEN_TSDB_SUB_SCAN\0201\022\017\n\013JSON_WR" + "ITER\0202\022\026\n\022HTPPD_LOG_SUB_SCAN\0203\022\022\n\016IMAGE_" + "SUB_SCAN\0204\022\025\n\021SEQUENCE_SUB_SCAN\0205\022\023\n\017PAR" + "TITION_LIMIT\0206\022\023\n\017PCAPNG_SUB_SCAN\0207\022\022\n\016R" + "UNTIME_FILTER\0208\022\017\n\013ROWKEY_JOIN\0209\022\023\n\017SYSL" + "OG_SUB_SCAN\020:\022\030\n\024STATISTICS_AGGREGATE\020;\022" + "\020\n\014UNPIVOT_MAPS\020<\022\024\n\020STATISTICS_MERGE\020=\022" + "\021\n\rLTSV_SUB_SCAN\020>*g\n\nSaslStatus\022\020\n\014SASL" + "_UNKNOWN\020\000\022\016\n\nSASL_START\020\001\022\024\n\020SASL_IN_PR" + "OGRESS\020\002\022\020\n\014SASL_SUCCESS\020\003\022\017\n\013SASL_FAILE" + "D\020\004B.\n\033org.apache.drill.exec.protoB\rUser" + "BitSharedH\001" }; ::google::protobuf::DescriptorPool::InternalAddGeneratedFile( - descriptor, 5659); + descriptor, 5691); ::google::protobuf::MessageFactory::InternalRegisterGeneratedFile( "UserBitShared.proto", &protobuf_RegisterTypes); ::protobuf_Types_2eproto::AddDescriptors(); @@ -8684,6 +8687,7 @@ const int OperatorProfile::kProcessNanosFieldNumber; const int OperatorProfile::kPeakLocalMemoryAllocatedFieldNumber; const int OperatorProfile::kMetricFieldNumber; const int OperatorProfile::kWaitNanosFieldNumber; +const int OperatorProfile::kOptimalMemAllocationFieldNumber; #endif // !defined(_MSC_VER) || _MSC_VER >= 1900 OperatorProfile::OperatorProfile() @@ -8701,15 +8705,15 @@ OperatorProfile::OperatorProfile(const OperatorProfile& from) metric_(from.metric_) { _internal_metadata_.MergeFrom(from._internal_metadata_); ::memcpy(&operator_id_, &from.operator_id_, - static_cast(reinterpret_cast(&wait_nanos_) - - reinterpret_cast(&operator_id_)) + sizeof(wait_nanos_)); + static_cast(reinterpret_cast(&optimal_mem_allocation_) - + reinterpret_cast(&operator_id_)) + sizeof(optimal_mem_allocation_)); // @@protoc_insertion_point(copy_constructor:exec.shared.OperatorProfile) } void OperatorProfile::SharedCtor() { ::memset(&operator_id_, 0, static_cast( - reinterpret_cast(&wait_nanos_) - - reinterpret_cast(&operator_id_)) + sizeof(wait_nanos_)); + reinterpret_cast(&optimal_mem_allocation_) - + reinterpret_cast(&operator_id_)) + sizeof(optimal_mem_allocation_)); } OperatorProfile::~OperatorProfile() { @@ -8743,10 +8747,10 @@ void OperatorProfile::Clear() { input_profile_.Clear(); metric_.Clear(); cached_has_bits = _has_bits_[0]; - if (cached_has_bits & 63u) { + if (cached_has_bits & 127u) { ::memset(&operator_id_, 0, static_cast( - reinterpret_cast(&wait_nanos_) - - reinterpret_cast(&operator_id_)) + sizeof(wait_nanos_)); + reinterpret_cast(&optimal_mem_allocation_) - + reinterpret_cast(&operator_id_)) + sizeof(optimal_mem_allocation_)); } _has_bits_.Clear(); _internal_metadata_.Clear(); @@ -8870,6 +8874,20 @@ bool OperatorProfile::MergePartialFromCodedStream( break; } + // optional int64 optimal_mem_allocation = 10; + case 10: { + if (static_cast< ::google::protobuf::uint8>(tag) == + static_cast< ::google::protobuf::uint8>(80u /* 80 & 0xFF */)) { + set_has_optimal_mem_allocation(); + DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive< + ::google::protobuf::int64, ::google::protobuf::internal::WireFormatLite::TYPE_INT64>( + input, &optimal_mem_allocation_))); + } else { + goto handle_unusual; + } + break; + } + default: { handle_unusual: if (tag == 0) { @@ -8945,6 +8963,11 @@ void OperatorProfile::SerializeWithCachedSizes( ::google::protobuf::internal::WireFormatLite::WriteInt64(9, this->wait_nanos(), output); } + // optional int64 optimal_mem_allocation = 10; + if (cached_has_bits & 0x00000040u) { + ::google::protobuf::internal::WireFormatLite::WriteInt64(10, this->optimal_mem_allocation(), output); + } + if (_internal_metadata_.have_unknown_fields()) { ::google::protobuf::internal::WireFormat::SerializeUnknownFields( _internal_metadata_.unknown_fields(), output); @@ -9006,6 +9029,11 @@ ::google::protobuf::uint8* OperatorProfile::InternalSerializeWithCachedSizesToAr target = ::google::protobuf::internal::WireFormatLite::WriteInt64ToArray(9, this->wait_nanos(), target); } + // optional int64 optimal_mem_allocation = 10; + if (cached_has_bits & 0x00000040u) { + target = ::google::protobuf::internal::WireFormatLite::WriteInt64ToArray(10, this->optimal_mem_allocation(), target); + } + if (_internal_metadata_.have_unknown_fields()) { target = ::google::protobuf::internal::WireFormat::SerializeUnknownFieldsToArray( _internal_metadata_.unknown_fields(), target); @@ -9045,7 +9073,7 @@ size_t OperatorProfile::ByteSizeLong() const { } } - if (_has_bits_[0 / 32] & 63u) { + if (_has_bits_[0 / 32] & 127u) { // optional int32 operator_id = 3; if (has_operator_id()) { total_size += 1 + @@ -9088,6 +9116,13 @@ size_t OperatorProfile::ByteSizeLong() const { this->wait_nanos()); } + // optional int64 optimal_mem_allocation = 10; + if (has_optimal_mem_allocation()) { + total_size += 1 + + ::google::protobuf::internal::WireFormatLite::Int64Size( + this->optimal_mem_allocation()); + } + } int cached_size = ::google::protobuf::internal::ToCachedSize(total_size); SetCachedSize(cached_size); @@ -9119,7 +9154,7 @@ void OperatorProfile::MergeFrom(const OperatorProfile& from) { input_profile_.MergeFrom(from.input_profile_); metric_.MergeFrom(from.metric_); cached_has_bits = from._has_bits_[0]; - if (cached_has_bits & 63u) { + if (cached_has_bits & 127u) { if (cached_has_bits & 0x00000001u) { operator_id_ = from.operator_id_; } @@ -9138,6 +9173,9 @@ void OperatorProfile::MergeFrom(const OperatorProfile& from) { if (cached_has_bits & 0x00000020u) { wait_nanos_ = from.wait_nanos_; } + if (cached_has_bits & 0x00000040u) { + optimal_mem_allocation_ = from.optimal_mem_allocation_; + } _has_bits_[0] |= cached_has_bits; } } @@ -9174,6 +9212,7 @@ void OperatorProfile::InternalSwap(OperatorProfile* other) { swap(process_nanos_, other->process_nanos_); swap(peak_local_memory_allocated_, other->peak_local_memory_allocated_); swap(wait_nanos_, other->wait_nanos_); + swap(optimal_mem_allocation_, other->optimal_mem_allocation_); swap(_has_bits_[0], other->_has_bits_[0]); _internal_metadata_.Swap(&other->_internal_metadata_); } diff --git a/contrib/native/client/src/protobuf/UserBitShared.pb.h b/contrib/native/client/src/protobuf/UserBitShared.pb.h index e186f138061..1d1e2ffddb1 100644 --- a/contrib/native/client/src/protobuf/UserBitShared.pb.h +++ b/contrib/native/client/src/protobuf/UserBitShared.pb.h @@ -3504,6 +3504,13 @@ class OperatorProfile : public ::google::protobuf::Message /* @@protoc_insertion ::google::protobuf::int64 wait_nanos() const; void set_wait_nanos(::google::protobuf::int64 value); + // optional int64 optimal_mem_allocation = 10; + bool has_optimal_mem_allocation() const; + void clear_optimal_mem_allocation(); + static const int kOptimalMemAllocationFieldNumber = 10; + ::google::protobuf::int64 optimal_mem_allocation() const; + void set_optimal_mem_allocation(::google::protobuf::int64 value); + // @@protoc_insertion_point(class_scope:exec.shared.OperatorProfile) private: void set_has_operator_id(); @@ -3518,6 +3525,8 @@ class OperatorProfile : public ::google::protobuf::Message /* @@protoc_insertion void clear_has_peak_local_memory_allocated(); void set_has_wait_nanos(); void clear_has_wait_nanos(); + void set_has_optimal_mem_allocation(); + void clear_has_optimal_mem_allocation(); ::google::protobuf::internal::InternalMetadataWithArena _internal_metadata_; ::google::protobuf::internal::HasBits<1> _has_bits_; @@ -3530,6 +3539,7 @@ class OperatorProfile : public ::google::protobuf::Message /* @@protoc_insertion ::google::protobuf::int64 process_nanos_; ::google::protobuf::int64 peak_local_memory_allocated_; ::google::protobuf::int64 wait_nanos_; + ::google::protobuf::int64 optimal_mem_allocation_; friend struct ::protobuf_UserBitShared_2eproto::TableStruct; }; // ------------------------------------------------------------------- @@ -8080,6 +8090,30 @@ inline void OperatorProfile::set_wait_nanos(::google::protobuf::int64 value) { // @@protoc_insertion_point(field_set:exec.shared.OperatorProfile.wait_nanos) } +// optional int64 optimal_mem_allocation = 10; +inline bool OperatorProfile::has_optimal_mem_allocation() const { + return (_has_bits_[0] & 0x00000040u) != 0; +} +inline void OperatorProfile::set_has_optimal_mem_allocation() { + _has_bits_[0] |= 0x00000040u; +} +inline void OperatorProfile::clear_has_optimal_mem_allocation() { + _has_bits_[0] &= ~0x00000040u; +} +inline void OperatorProfile::clear_optimal_mem_allocation() { + optimal_mem_allocation_ = GOOGLE_LONGLONG(0); + clear_has_optimal_mem_allocation(); +} +inline ::google::protobuf::int64 OperatorProfile::optimal_mem_allocation() const { + // @@protoc_insertion_point(field_get:exec.shared.OperatorProfile.optimal_mem_allocation) + return optimal_mem_allocation_; +} +inline void OperatorProfile::set_optimal_mem_allocation(::google::protobuf::int64 value) { + set_has_optimal_mem_allocation(); + optimal_mem_allocation_ = value; + // @@protoc_insertion_point(field_set:exec.shared.OperatorProfile.optimal_mem_allocation) +} + // ------------------------------------------------------------------- // StreamProfile diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OpProfileDef.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OpProfileDef.java index 8768eb34ce1..d9746dcd6e3 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OpProfileDef.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OpProfileDef.java @@ -22,11 +22,13 @@ public class OpProfileDef { public int operatorId; public int operatorType; public int incomingCount; + public long optimalMemoryAllocation; - public OpProfileDef(int operatorId, int operatorType, int incomingCount) { + public OpProfileDef(int operatorId, int operatorType, int incomingCount, long optimalMemoryAllocation) { this.operatorId = operatorId; this.operatorType = operatorType; this.incomingCount = incomingCount; + this.optimalMemoryAllocation = optimalMemoryAllocation; } public int getOperatorId(){ return operatorId; diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java index d47e8d9c00f..c512959dc79 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java @@ -57,7 +57,7 @@ public OperatorContextImpl(PhysicalOperator popConfig, FragmentContextImpl conte } else { OpProfileDef def = new OpProfileDef(popConfig.getOperatorId(), popConfig.getOperatorType(), - OperatorUtilities.getChildCount(popConfig)); + OperatorUtilities.getChildCount(popConfig), popConfig.getMaxAllocation()); this.stats = context.getStats().newOperatorStats(def, allocator); } } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorStats.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorStats.java index 67a8b80f280..f682104aa1a 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorStats.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorStats.java @@ -45,7 +45,7 @@ public class OperatorStats { public long[] recordsReceivedByInput; public long[] batchesReceivedByInput; private long[] schemaCountByInput; - + private long optimalMemoryAllocation; private boolean inProcessing = false; private boolean inSetup = false; @@ -62,7 +62,7 @@ public class OperatorStats { private int inputCount; public OperatorStats(OpProfileDef def, BufferAllocator allocator){ - this(def.getOperatorId(), def.getOperatorType(), def.getIncomingCount(), allocator); + this(def.getOperatorId(), def.getOperatorType(), def.getIncomingCount(), allocator, def.optimalMemoryAllocation); } /** @@ -74,7 +74,7 @@ public OperatorStats(OpProfileDef def, BufferAllocator allocator){ */ public OperatorStats(OperatorStats original, boolean isClean) { - this(original.operatorId, original.operatorType, original.inputCount, original.allocator); + this(original.operatorId, original.operatorType, original.inputCount, original.allocator, original.optimalMemoryAllocation); if ( !isClean ) { inProcessing = original.inProcessing; @@ -88,7 +88,7 @@ public OperatorStats(OperatorStats original, boolean isClean) { } @VisibleForTesting - public OperatorStats(int operatorId, int operatorType, int inputCount, BufferAllocator allocator) { + public OperatorStats(int operatorId, int operatorType, int inputCount, BufferAllocator allocator, long initialAllocation) { super(); this.allocator = allocator; this.operatorId = operatorId; @@ -97,6 +97,7 @@ public OperatorStats(int operatorId, int operatorType, int inputCount, BufferAll this.recordsReceivedByInput = new long[inputCount]; this.batchesReceivedByInput = new long[inputCount]; this.schemaCountByInput = new long[inputCount]; + this.optimalMemoryAllocation = initialAllocation; } private String assertionError(String msg){ @@ -207,6 +208,7 @@ public OperatorProfile getProfile() { .setOperatorId(operatorId) // .setSetupNanos(setupNanos) // .setProcessNanos(processingNanos) + .setOptimalMemAllocation(optimalMemoryAllocation) .setWaitNanos(waitNanos); if (allocator != null) { diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScan.java index 126ad0751d4..07693c4c9c8 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScan.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScan.java @@ -101,18 +101,6 @@ public boolean enforceWidth() { return getMinParallelizationWidth() > 1; } - @Override - @JsonIgnore - public long getInitialAllocation() { - return 0; - } - - @Override - @JsonIgnore - public long getMaxAllocation() { - return 0; - } - @Override @JsonIgnore public boolean canPushdownProjects(List columns) { diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/BaseRootExec.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/BaseRootExec.java index 95a1235017a..9c1ba591435 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/BaseRootExec.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/BaseRootExec.java @@ -54,7 +54,7 @@ public BaseRootExec(final RootFragmentContext fragmentContext, final OperatorCon } //Creating new stat for appending to list stats = new OperatorStats(new OpProfileDef(config.getOperatorId(), - config.getOperatorType(), OperatorUtilities.getChildCount(config)), + config.getOperatorType(), OperatorUtilities.getChildCount(config), config.getMaxAllocation()), this.oContext.getAllocator()); fragmentContext.getStats().addOperatorStats(this.stats); this.fragmentContext = fragmentContext; 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 fecea5e7f6e..541130b1670 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 @@ -17,6 +17,7 @@ */ package org.apache.drill.exec.planner.fragment; +import com.fasterxml.jackson.core.JsonProcessingException; import org.apache.commons.lang3.tuple.Pair; import org.apache.drill.common.exceptions.ExecutionSetupException; import org.apache.drill.common.util.function.CheckedConsumer; @@ -29,7 +30,7 @@ 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 com.fasterxml.jackson.databind.ObjectMapper; import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; @@ -46,6 +47,7 @@ * fragment is based on the cluster state and provided queue configuration. */ public class DistributedQueueParallelizer extends SimpleParallelizer { + static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DistributedQueueParallelizer.class); private final boolean planHasMemory; private final QueryContext queryContext; private final QueryResourceManager rm; @@ -65,9 +67,13 @@ public BiFunction getMemory() { if (!planHasMemory) { final DrillNode drillEndpointNode = DrillNode.create(endpoint); if (operator.isBufferedOperator(queryContext)) { - return operators.get(drillEndpointNode).get(operator); + Long operatorsMemory = operators.get(drillEndpointNode).get(operator); + logger.debug(" Memory requirement for the operator {} in endpoint {} is {}", operator, endpoint, operatorsMemory); + return operatorsMemory; } else { - return operator.getMaxAllocation(); + Long nonBufferedMemory = (long)operator.getCost().getMemoryCost(); + logger.debug(" Memory requirement for the operator {} in endpoint {} is {}", operator, endpoint, nonBufferedMemory); + return nonBufferedMemory; } } else { @@ -92,10 +98,11 @@ public BiFunction getMemory() { */ public void adjustMemory(PlanningSet planningSet, Set roots, Map onlineEndpointUUIDs) throws ExecutionSetupException { - if (planHasMemory) { + logger.debug(" Plan already has memory settings. Adjustment of the memory is skipped"); return; } + logger.info(" Memory adjustment phase triggered"); final Map onlineDrillNodeUUIDs = onlineEndpointUUIDs.entrySet().stream() .collect(Collectors.toMap(x -> DrillNode.create(x.getKey()), x -> x.getValue())); @@ -112,7 +119,7 @@ public void adjustMemory(PlanningSet planningSet, Set roots, for (Wrapper wrapper : roots) { traverse(wrapper, CheckedConsumer.throwingConsumerWrapper((Wrapper fragment) -> { - MemoryCalculator calculator = new MemoryCalculator(planningSet, queryContext); + MemoryCalculator calculator = new MemoryCalculator(planningSet, queryContext, rm.minimumOperatorMemory()); fragment.getNode().getRoot().accept(calculator, fragment); NodeResources.merge(totalNodeResources, fragment.getResourceMap()); operators.entrySet() @@ -122,6 +129,10 @@ public void adjustMemory(PlanningSet planningSet, Set roots, })); } + if (logger.isDebugEnabled()) { + logger.debug(" Total node resource requirements for the plan is {}", getJSONFromResourcesMap(totalNodeResources)); + } + final QueryQueueConfig queueConfig; try { queueConfig = this.rm.selectQueue(max(totalNodeResources.values())); @@ -130,8 +141,10 @@ public void adjustMemory(PlanningSet planningSet, Set roots, } Map>> memoryAdjustedOperators = ensureOperatorMemoryWithinLimits(operators, totalNodeResources, - queueConfig.getMaxQueryMemoryInMBPerNode()); + List>> memoryAdjustedOperators = + ensureOperatorMemoryWithinLimits(operators, totalNodeResources, + convertMBToBytes(Math.min(queueConfig.getMaxQueryMemoryInMBPerNode(), + queueConfig.getQueueTotalMemoryInMB(onlineEndpointUUIDs.size())))); memoryAdjustedOperators.entrySet().stream().forEach((x) -> { Map memoryPerOperator = x.getValue().stream() .collect(Collectors.toMap(operatorLongPair -> operatorLongPair.getLeft(), @@ -140,9 +153,17 @@ public void adjustMemory(PlanningSet planningSet, Set roots, this.operators.put(x.getKey(), memoryPerOperator); }); + if (logger.isDebugEnabled()) { + logger.debug(" Total node resource requirements after adjustment {}", getJSONFromResourcesMap(totalNodeResources)); + } + this.rm.setCost(convertToUUID(totalNodeResources, onlineDrillNodeUUIDs)); } + private long convertMBToBytes(long value) { + return value * 1024 * 1024; + } + private Map convertToUUID(Map nodeResourcesMap, Map onlineDrillNodeUUIDs) { Map nodeResourcesPerUUID = new HashMap<>(); @@ -172,50 +193,81 @@ private NodeResources max(Collection resources) { */ private Map>> ensureOperatorMemoryWithinLimits(Map>> memoryPerOperator, - Map nodeResourceMap, long nodeLimit) { + Map nodeResourceMap, long nodeLimit) throws ExecutionSetupException { // Get the physical operators which are above the node memory limit. - Map>> onlyMemoryAboveLimitOperators = new HashMap<>(); - memoryPerOperator.entrySet().stream().forEach((entry) -> { - onlyMemoryAboveLimitOperators.putIfAbsent(entry.getKey(), new ArrayList<>()); - if (nodeResourceMap.get(entry.getKey()).getMemoryInBytes() > nodeLimit) { - onlyMemoryAboveLimitOperators.get(entry.getKey()).addAll(entry.getValue()); - } - }); - + Map>> onlyMemoryAboveLimitOperators = memoryPerOperator.entrySet() + .stream() + .filter(entry -> nodeResourceMap.get(entry.getKey()).getMemoryInBytes() > nodeLimit) + .collect(Collectors.toMap(entry -> entry.getKey(), entry -> entry.getValue())); // 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<>(); onlyMemoryAboveLimitOperators.entrySet().stream().forEach( - entry -> { - Long totalMemory = entry.getValue().stream().mapToLong(Pair::getValue).sum(); - List> adjustedMemory = entry.getValue().stream().map(operatorMemory -> { + CheckedConsumer.throwingConsumerWrapper(entry -> { + Long totalBufferedOperatorsMemoryReq = entry.getValue().stream().mapToLong(Pair::getValue).sum(); + Long nonBufferedOperatorsMemoryReq = nodeResourceMap.get(entry.getKey()).getMemoryInBytes() - totalBufferedOperatorsMemoryReq; + Long bufferedOperatorsMemoryLimit = nodeLimit - nonBufferedOperatorsMemoryReq; + if (bufferedOperatorsMemoryLimit < 0 || nonBufferedOperatorsMemoryReq < 0) { + logger.error(" Operator memory requirements for buffered operators {} or non buffered operators {} is negative", bufferedOperatorsMemoryLimit, + nonBufferedOperatorsMemoryReq); + throw new ExecutionSetupException("Operator memory requirements for buffered operators " + bufferedOperatorsMemoryLimit + " or non buffered operators " + + nonBufferedOperatorsMemoryReq + " is less than zero"); + } + List> adjustedMemory = entry.getValue().stream().map(operatorAndMemory -> { // formula to adjust the memory is (optimalMemory / totalMemory(this is for all the operators)) * permissible_node_limit. - return Pair.of(operatorMemory.getKey(), (long) Math.ceil(operatorMemory.getValue()/totalMemory * nodeLimit)); + return Pair.of(operatorAndMemory.getKey(), + Math.max(this.rm.minimumOperatorMemory(), + (long) Math.ceil(operatorAndMemory.getValue()/totalBufferedOperatorsMemoryReq * bufferedOperatorsMemoryLimit))); }).collect(Collectors.toList()); memoryAdjustedDrillbits.put(entry.getKey(), adjustedMemory); NodeResources nodeResources = nodeResourceMap.get(entry.getKey()); - nodeResources.setMemoryInBytes(adjustedMemory.stream().mapToLong(Pair::getValue).sum()); - } + nodeResources.setMemoryInBytes(nonBufferedOperatorsMemoryReq + adjustedMemory.stream().mapToLong(Pair::getValue).sum()); + }) ); + checkIfWithinLimit(nodeResourceMap, nodeLimit); + // 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<>(); - memoryPerOperator.entrySet().stream().filter((entry) -> !memoryAdjustedDrillbits.containsKey(entry.getKey())).forEach( - operatorMemory -> { - allDrillbits.put(operatorMemory.getKey(), operatorMemory.getValue()); - } - ); + Map>> allDrillbits = memoryPerOperator.entrySet() + .stream() + .filter((entry) -> !memoryAdjustedDrillbits.containsKey(entry.getKey())) + .collect(Collectors.toMap(entry -> entry.getKey(), entry -> entry.getValue())); memoryAdjustedDrillbits.entrySet().stream().forEach( - operatorMemory -> { - allDrillbits.put(operatorMemory.getKey(), operatorMemory.getValue()); - } - ); + operatorMemory -> allDrillbits.put(operatorMemory.getKey(), operatorMemory.getValue())); // At this point allDrillbits contains the operators on all drillbits. The memory also is adjusted based on the nodeLimit and // the ratio of their requirements. return allDrillbits; } + + private void checkIfWithinLimit(Map nodeResourcesMap, long nodeLimit) throws ExecutionSetupException { + for (Map.Entry entry : nodeResourcesMap.entrySet()) { + if (entry.getValue().getMemoryInBytes() > nodeLimit) { + logger.error(" Memory requirement for the query cannot be adjusted." + + " Memory requirement {} (in bytes) for a node {} is greater than limit {}", entry.getValue() + .getMemoryInBytes(), entry.getKey(), nodeLimit); + throw new ExecutionSetupException("Minimum memory requirement " + + entry.getValue().getMemoryInBytes() + " for a node " + entry.getKey() + " is greater than limit: " + nodeLimit); + } + } + } + + private String getJSONFromResourcesMap(Map resourcesMap) { + String json = ""; + try { + json = new ObjectMapper().writeValueAsString(resourcesMap.entrySet() + .stream() + .collect(Collectors.toMap(entry -> entry.getKey() + .toString(), Map.Entry::getValue))); + } catch (JsonProcessingException exception) { + logger.error(" Cannot convert the Node resources map to json "); + } + + return json; + } } \ No newline at end of file diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Fragment.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Fragment.java index 0212e088211..3d28067d563 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Fragment.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Fragment.java @@ -17,19 +17,17 @@ */ package org.apache.drill.exec.planner.fragment; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; - import org.apache.drill.exec.ops.QueryContext; import org.apache.drill.exec.physical.base.AbstractPhysicalVisitor; import org.apache.drill.exec.physical.base.Exchange; import org.apache.drill.exec.physical.base.PhysicalOperator; -import org.apache.drill.exec.util.memory.ZKQueueMemoryAllocationUtilities; import org.apache.drill.exec.work.foreman.ForemanSetupException; - import org.apache.drill.shaded.guava.com.google.common.collect.Lists; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + public class Fragment implements Iterable { static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Fragment.class); diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Materializer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Materializer.java index fdfa95c393a..d806b52e20a 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Materializer.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Materializer.java @@ -55,6 +55,7 @@ public PhysicalOperator visitExchange(Exchange exchange, IndexedFragmentNode iNo PhysicalOperator materializedSender = exchange.getSender(iNode.getMinorFragmentId(), child); materializedSender.setOperatorId(0); materializedSender.setCost(exchange.getCost()); + materializedSender.setMaxAllocation(exchange.getMaxAllocation()); // logger.debug("Visit sending exchange, materialized {} with child {}.", materializedSender, child); return materializedSender; @@ -62,6 +63,7 @@ public PhysicalOperator visitExchange(Exchange exchange, IndexedFragmentNode iNo // receiving exchange. PhysicalOperator materializedReceiver = exchange.getReceiver(iNode.getMinorFragmentId()); materializedReceiver.setOperatorId(Short.MAX_VALUE & exchange.getOperatorId()); + materializedReceiver.setMaxAllocation(exchange.getMaxAllocation()); // logger.debug("Visit receiving exchange, materialized receiver: {}.", materializedReceiver); materializedReceiver.setCost(exchange.getCost()); return materializedReceiver; @@ -70,8 +72,10 @@ public PhysicalOperator visitExchange(Exchange exchange, IndexedFragmentNode iNo @Override public PhysicalOperator visitGroupScan(GroupScan groupScan, IndexedFragmentNode iNode) throws ExecutionSetupException { + iNode.addAllocation(groupScan); SubScan child = groupScan.getSpecificScan(iNode.getMinorFragmentId()); child.setOperatorId(Short.MAX_VALUE & groupScan.getOperatorId()); + child.setMaxAllocation(groupScan.getMaxAllocation()); // remember the subscan for future use iNode.addSubScan(child); return child; @@ -89,11 +93,11 @@ public PhysicalOperator visitSubScan(SubScan subScan, IndexedFragmentNode value) @Override public PhysicalOperator visitStore(Store store, IndexedFragmentNode iNode) throws ExecutionSetupException { PhysicalOperator child = store.getChild().accept(this, iNode); - iNode.addAllocation(store); try { PhysicalOperator o = store.getSpecificStore(child, iNode.getMinorFragmentId()); + o.setMaxAllocation(store.getMaxAllocation()); o.setOperatorId(Short.MAX_VALUE & store.getOperatorId()); // logger.debug("New materialized store node {} with child {}", o, child); return o; @@ -112,6 +116,7 @@ public PhysicalOperator visitOp(PhysicalOperator op, IndexedFragmentNode iNode) } PhysicalOperator newOp = op.getNewWithChildren(children); newOp.setCost(op.getCost()); + newOp.setMaxAllocation(op.getMaxAllocation()); newOp.setOperatorId(Short.MAX_VALUE & op.getOperatorId()); return newOp; } @@ -128,6 +133,7 @@ public PhysicalOperator visitLateralJoin(LateralJoinPOP op, IndexedFragmentNode PhysicalOperator newOp = op.getNewWithChildren(children); newOp.setCost(op.getCost()); + newOp.setMaxAllocation(op.getMaxAllocation()); newOp.setOperatorId(Short.MAX_VALUE & op.getOperatorId()); ((LateralJoinPOP) newOp).setUnnestForLateralJoin(unnestForThisLateral); @@ -138,6 +144,7 @@ public PhysicalOperator visitLateralJoin(LateralJoinPOP op, IndexedFragmentNode public PhysicalOperator visitUnnest(UnnestPOP unnest, IndexedFragmentNode value) throws ExecutionSetupException { PhysicalOperator newOp = visitOp(unnest, value); value.addUnnest((UnnestPOP) newOp); + newOp.setMaxAllocation(unnest.getMaxAllocation()); return newOp; } @@ -157,6 +164,7 @@ public PhysicalOperator visitRowKeyJoin(RowKeyJoinPOP op, IndexedFragmentNode iN PhysicalOperator newOp = op.getNewWithChildren(children); newOp.setCost(op.getCost()); newOp.setOperatorId(Short.MAX_VALUE & op.getOperatorId()); + newOp.setMaxAllocation(op.getMaxAllocation()); ((RowKeyJoinPOP)newOp).setSubScanForRowKeyJoin(subScanInLeftInput); 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 d3d759ca437..4593c55f6ad 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 @@ -49,11 +49,13 @@ public class MemoryCalculator extends AbstractOpWrapperVisitor>> bufferedOperators; private final QueryContext queryContext; + private final long MINIMUM_MEMORY_FOR_BUFFER_OPERS; - public MemoryCalculator(PlanningSet planningSet, QueryContext context) { + public MemoryCalculator(PlanningSet planningSet, QueryContext context, long minMemory) { this.planningSet = planningSet; this.bufferedOperators = new HashMap<>(); this.queryContext = context; + this.MINIMUM_MEMORY_FOR_BUFFER_OPERS = minMemory; } // Helper method to compute the minor fragment count per drillbit. This method returns @@ -86,7 +88,7 @@ public Void visitSendingExchange(Exchange exchange, Wrapper fragment) throws Run getMinorFragCountPerDrillbit(fragment), // get the memory requirements for the sender operator. (x) -> exchange.getSenderMemory(receivingFragment.getWidth(), x.getValue())); - return visitOp(exchange, fragment); + return visit(exchange, fragment); } @Override @@ -117,19 +119,26 @@ public Void visitReceivingExchange(Exchange exchange, Wrapper fragment) throws R return null; } + private Void visit(PhysicalOperator op, Wrapper fragment) { + for (PhysicalOperator child : op) { + child.accept(this, fragment); + } + return null; + } + public List> getBufferedOperators(DrillNode endpoint) { return this.bufferedOperators.getOrDefault(endpoint, new ArrayList<>()); } @Override public Void visitOp(PhysicalOperator op, Wrapper fragment) { - long memoryCost = (int)Math.ceil(op.getCost().getMemoryCost()); + long memoryCost = (long)Math.ceil(op.getCost().getMemoryCost()); if (op.isBufferedOperator(queryContext)) { // If the operator is a buffered operator then get the memory estimates of the optimizer. // The memory estimates of the optimizer are for the whole operator spread across all the // 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()); + long memoryCostPerMinorFrag = Math.max((long)Math.ceil(memoryCost/fragment.getAssignedEndpoints().size()), MINIMUM_MEMORY_FOR_BUFFER_OPERS); Map drillbitEndpointMinorFragMap = getMinorFragCountPerDrillbit(fragment); Map x.getKey(), (x) -> Pair.of(op, - memoryCostPerMinorFrag * x.getValue()))); + memoryCostPerMinorFrag))); bufferedOperatorsPerDrillbit.entrySet().forEach((x) -> { bufferedOperators.putIfAbsent(x.getKey(), new ArrayList<>()); bufferedOperators.get(x.getKey()).add(x.getValue()); @@ -153,10 +162,7 @@ public Void visitOp(PhysicalOperator op, Wrapper fragment) { getMinorFragCountPerDrillbit(fragment), (x) -> memoryCost * x.getValue()); } - for (PhysicalOperator child : op) { - child.accept(this, fragment); - } - return null; + return visit(op, fragment); } } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/ZKQueueParallelizer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/ZKQueueParallelizer.java index 6e529224b9d..98fb2b32343 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/ZKQueueParallelizer.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/ZKQueueParallelizer.java @@ -64,7 +64,6 @@ public void adjustMemory(PlanningSet planningSet, Set roots, } endpointMap = collector.getNodeMap(); - ZKQueueMemoryAllocationUtilities.planMemory(queryContext, this.resourceManager, endpointMap); } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DirectPlan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DirectPlan.java index 3e1d6c79832..71fb70ebb18 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DirectPlan.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DirectPlan.java @@ -21,15 +21,18 @@ import org.apache.drill.common.logical.PlanProperties.Generator.ResultMode; import org.apache.drill.common.logical.PlanProperties.PlanPropertiesBuilder; import org.apache.drill.common.logical.PlanProperties.PlanType; +import org.apache.drill.exec.ExecConstants; import org.apache.drill.exec.ops.QueryContext; import org.apache.drill.exec.physical.PhysicalPlan; +import org.apache.drill.exec.physical.base.PhysicalOperator; import org.apache.drill.exec.physical.config.Screen; +import org.apache.drill.exec.planner.cost.PrelCostEstimates; import org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler; import org.apache.drill.exec.planner.sql.handlers.SimpleCommandResult; -import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint; import org.apache.drill.exec.store.direct.DirectGroupScan; import org.apache.drill.exec.store.pojo.PojoRecordReader; +import java.util.Collection; import java.util.Collections; import java.util.List; @@ -43,20 +46,24 @@ public static PhysicalPlan createDirectPlan(QueryContext context, boolean result @SuppressWarnings("unchecked") public static PhysicalPlan createDirectPlan(QueryContext context, T obj){ - return createDirectPlan(context.getCurrentEndpoint(), Collections.singletonList(obj), (Class) obj.getClass()); + return createDirectPlan(context, Collections.singletonList(obj), (Class) obj.getClass()); } - public static PhysicalPlan createDirectPlan(DrillbitEndpoint endpoint, List records, Class clazz){ + public static PhysicalPlan createDirectPlan(QueryContext context, List records, Class clazz){ PojoRecordReader reader = new PojoRecordReader<>(clazz, records); DirectGroupScan scan = new DirectGroupScan(reader); - Screen screen = new Screen(scan, endpoint); + Screen screen = new Screen(scan, context.getCurrentEndpoint()); PlanPropertiesBuilder propsBuilder = PlanProperties.builder(); propsBuilder.type(PlanType.APACHE_DRILL_PHYSICAL); propsBuilder.version(1); propsBuilder.resultMode(ResultMode.EXEC); propsBuilder.generator(DirectPlan.class.getSimpleName(), ""); + Collection pops = DefaultSqlHandler.getPops(screen); + for (PhysicalOperator pop : pops) { + pop.setCost(new PrelCostEstimates(context.getOptions().getLong(ExecConstants.OUTPUT_BATCH_SIZE), 0)); + } return new PhysicalPlan(propsBuilder.build(), DefaultSqlHandler.getPops(screen)); } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ShowFilesHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ShowFilesHandler.java index 1318e6f47de..337fbfd76b8 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ShowFilesHandler.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ShowFilesHandler.java @@ -92,7 +92,7 @@ public PhysicalPlan getPlan(SqlNode sqlNode) throws ForemanSetupException { .map(fileStatus -> new ShowFilesCommandResult(new Records.File(wsSchema.getFullSchemaName(), wsSchema, fileStatus))) .collect(Collectors.toList()); - return DirectPlan.createDirectPlan(context.getCurrentEndpoint(), records, ShowFilesCommandResult.class); + return DirectPlan.createDirectPlan(context, records, ShowFilesCommandResult.class); } /** 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 befa4bce838..b140abd2a26 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 @@ -404,7 +404,7 @@ private String updateBlobs(Map resourcesMap, QueryQueueCo throw new RMBlobUpdateException(String.format("Failed to update the cluster state blob and queue blob in a " + "transaction. [Details: %s]", exceptionStringBuilder.toString())); } - logger.debug("Successfully updated the blobs in a transaction. [Details: %s]", exceptionStringBuilder.toString()); + logger.debug("Successfully updated the blobs in a transaction. [Details: {}]", exceptionStringBuilder.toString()); // Reset the exceptionStringBuilder for next event exceptionStringBuilder.delete(0, exceptionStringBuilder.length()); 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 aad01c0eb16..4899715e674 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 @@ -425,6 +425,7 @@ private void runPhysicalPlan(final PhysicalPlan plan) throws ExecutionSetupExcep private void runPhysicalPlan(final PhysicalPlan plan, Pointer textPlan) throws ExecutionSetupException { validatePlan(plan); + queryRM.setCost(plan.totalCost()); final QueryWorkUnit work = getQueryWorkUnit(plan, queryRM); if (enableRuntimeFilter) { runtimeFilterRouter = new RuntimeFilterRouter(work, drillbitContext); @@ -433,7 +434,6 @@ private void runPhysicalPlan(final PhysicalPlan plan, Pointer textPlan) if (textPlan != null) { queryManager.setPlanText(textPlan.value); } - queryManager.setTotalCost(plan.totalCost()); work.applyPlan(drillbitContext.getPlanReader()); logWorkUnit(work); 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 c0cdefe21aa..96b06912dbb 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 @@ -273,7 +273,6 @@ public boolean hasQueue() { @Override public void setCost(double cost) { - throw new UnsupportedOperationException("DistributedQueryRM doesn't support cost in double format"); } public void setCost(Map costOnAssignedEndpoints) { diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/memory/TestAllocators.java b/exec/java-exec/src/test/java/org/apache/drill/exec/memory/TestAllocators.java index 4fb5a8f4417..d49cfd4f0ff 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/memory/TestAllocators.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/memory/TestAllocators.java @@ -220,7 +220,7 @@ public void testAllocators() throws Exception { // Use some bogus operator type to create a new operator context. def = new OpProfileDef(physicalOperator1.getOperatorId(), UserBitShared.CoreOperatorType.MOCK_SUB_SCAN_VALUE, - OperatorUtilities.getChildCount(physicalOperator1)); + OperatorUtilities.getChildCount(physicalOperator1), physicalOperator1.getMaxAllocation()); stats = fragmentContext1.getStats().newOperatorStats(def, fragmentContext1.getAllocator()); // Add a couple of Operator Contexts @@ -234,7 +234,7 @@ public void testAllocators() throws Exception { OperatorContext oContext21 = fragmentContext1.newOperatorContext(physicalOperator3); def = new OpProfileDef(physicalOperator4.getOperatorId(), UserBitShared.CoreOperatorType.TEXT_WRITER_VALUE, - OperatorUtilities.getChildCount(physicalOperator4)); + OperatorUtilities.getChildCount(physicalOperator4), physicalOperator4.getMaxAllocation()); stats = fragmentContext2.getStats().newOperatorStats(def, fragmentContext2.getAllocator()); OperatorContext oContext22 = fragmentContext2.newOperatorContext(physicalOperator4, stats); DrillBuf b22 = oContext22.getAllocator().buffer(2000000); @@ -248,7 +248,7 @@ public void testAllocators() throws Exception { // New fragment starts an operator that allocates an amount within the limit def = new OpProfileDef(physicalOperator5.getOperatorId(), UserBitShared.CoreOperatorType.UNION_VALUE, - OperatorUtilities.getChildCount(physicalOperator5)); + OperatorUtilities.getChildCount(physicalOperator5), physicalOperator5.getMaxAllocation()); stats = fragmentContext3.getStats().newOperatorStats(def, fragmentContext3.getAllocator()); OperatorContext oContext31 = fragmentContext3.newOperatorContext(physicalOperator5, stats); diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestExternalSortInternals.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestExternalSortInternals.java index 9c2d5d8feca..b212e76b8dd 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestExternalSortInternals.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestExternalSortInternals.java @@ -655,7 +655,7 @@ public void testMergeLimit() { @Test public void testMetrics() { - OperatorStats stats = new OperatorStats(100, 101, 0, fixture.allocator()); + OperatorStats stats = new OperatorStats(100, 101, 0, fixture.allocator(), 0); SortMetrics metrics = new SortMetrics(stats); // Input stats 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 cd6b0a9a280..577014eaafe 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 @@ -27,7 +27,7 @@ import org.apache.drill.exec.planner.fragment.PlanningSet; 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.common.DrillNode; import org.apache.drill.exec.pop.PopUnitTestBase; import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint; import org.apache.drill.exec.proto.UserBitShared; @@ -44,9 +44,10 @@ 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 org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; import java.util.ArrayList; import java.util.HashMap; @@ -59,6 +60,8 @@ import static org.junit.Assert.assertTrue; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -90,7 +93,7 @@ private static final DrillbitEndpoint newDrillbitEndpoint(String address, int po UserBitShared.QueryId.getDefaultInstance()); private static Map onlineEndpoints; - private Map resources; + private Map totalResources; @AfterClass public static void close() throws Exception { @@ -102,7 +105,16 @@ private QueryResourceManager mockResourceManager() throws QueueSelectionExceptio final QueryQueueConfig queueConfig = mock(QueryQueueConfig.class); when(queueConfig.getMaxQueryMemoryInMBPerNode()).thenReturn(10L); + when(queueConfig.getQueueTotalMemoryInMB(anyInt())).thenReturn(100L); when(mockRM.selectQueue(any(NodeResources.class))).thenReturn(queueConfig); + when(mockRM.minimumOperatorMemory()).thenReturn(40L); + doAnswer(new Answer() { + @Override + public Object answer(InvocationOnMock invocation) throws Throwable { + totalResources = (Map) invocation.getArguments()[0]; + return null; + } + }).when(mockRM).setCost(any(Map.class)); return mockRM; } @@ -116,7 +128,7 @@ private final Wrapper mockWrapper(Wrapper rootFragment, List mockdependencies = new ArrayList<>(); for (Wrapper dependency : rootFragment.getFragmentDependencies()) { - mockdependencies.add(mockWrapper(dependency, resourceMap, endpoints, originalToMockWrapper)); + mockdependencies.add(mockWrapper(dependency, getNodeResources(), endpoints, originalToMockWrapper)); } when(mockWrapper.getNode()).thenReturn(rootFragment.getNode()); @@ -129,11 +141,9 @@ private final Wrapper mockWrapper(Wrapper rootFragment, } private final PlanningSet mockPlanningSet(PlanningSet planningSet, - Map resourceMap, List endpoints) { Map wrapperToMockWrapper = new HashMap<>(); - Wrapper rootFragment = mockWrapper( planningSet.getRootWrapper(), resourceMap, - endpoints, wrapperToMockWrapper); + Wrapper rootFragment = mockWrapper(planningSet.getRootWrapper(), getNodeResources(), endpoints, wrapperToMockWrapper); PlanningSet mockPlanningSet = mock(PlanningSet.class); when(mockPlanningSet.getRootWrapper()).thenReturn(rootFragment); when(mockPlanningSet.get(any(Fragment.class))).thenAnswer(invocation -> { @@ -196,10 +206,13 @@ private Fragment getRootFragmentFromPlan(DrillbitContext context, } private PlanningSet preparePlanningSet(List activeEndpoints, long slice_target, - Map resources, String sql, - SimpleParallelizer parallelizer) throws Exception { + String sql, SimpleParallelizer parallelizer) throws Exception { Fragment rootFragment = getRootFragmentFromPlan(drillbitContext, getPlanForQuery(sql, 10, slice_target)); - return mockPlanningSet(parallelizer.prepareFragmentTree(rootFragment), resources, activeEndpoints); + return mockPlanningSet(parallelizer.prepareFragmentTree(rootFragment), activeEndpoints); + } + + private Map getNodeResources() { + return onlineEndpoints.keySet().stream().collect(Collectors.toMap(x -> DrillNode.create(x), x -> NodeResources.create())); } @BeforeClass @@ -207,21 +220,14 @@ 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 { String sql = "SELECT * from cp.`tpch/nation.parquet`"; SimpleParallelizer parallelizer = new DistributedQueueParallelizer(false, queryContext, mockResourceManager()); - PlanningSet planningSet = preparePlanningSet(new ArrayList<>(onlineEndpoints.keySet()), DEFAULT_SLICE_TARGET, resources, sql, parallelizer); + PlanningSet planningSet = preparePlanningSet(new ArrayList<>(onlineEndpoints.keySet()), DEFAULT_SLICE_TARGET, sql, parallelizer); parallelizer.adjustMemory(planningSet, createSet(planningSet.getRootWrapper()), onlineEndpoints); - assertTrue("memory requirement is different", Iterables.all(resources.entrySet(), (e) -> e.getValue().getMemoryInBytes() == 30)); + assertTrue("memory requirement is different", Iterables.all(totalResources.entrySet(), (e) -> e.getValue().getMemoryInBytes() == 30)); } @@ -230,21 +236,20 @@ public void TestSingleMajorFragmentWithGroupByProjectAndScan() throws Exception String sql = "SELECT dept_id, count(*) from cp.`tpch/lineitem.parquet` group by dept_id"; SimpleParallelizer parallelizer = new DistributedQueueParallelizer(false, queryContext, mockResourceManager()); - PlanningSet planningSet = preparePlanningSet(new ArrayList<>(onlineEndpoints.keySet()), DEFAULT_SLICE_TARGET, resources, sql, parallelizer); + PlanningSet planningSet = preparePlanningSet(new ArrayList<>(onlineEndpoints.keySet()), DEFAULT_SLICE_TARGET, sql, parallelizer); parallelizer.adjustMemory(planningSet, createSet(planningSet.getRootWrapper()), onlineEndpoints); - assertTrue("memory requirement is different", Iterables.all(resources.entrySet(), (e) -> e.getValue().getMemoryInBytes() == 529570)); + assertTrue("memory requirement is different", Iterables.all(totalResources.entrySet(), (e) -> e.getValue().getMemoryInBytes() == 529570)); } @Test - public void TestTwoMajorFragmentWithSortyProjectAndScan() throws Exception { + public void TestTwoMajorFragmentWithSortProjectAndScan() throws Exception { 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, sql, parallelizer); parallelizer.adjustMemory(planningSet, createSet(planningSet.getRootWrapper()), onlineEndpoints); - assertTrue("memory requirement is different", Iterables.all(resources.entrySet(), (e) -> e.getValue().getMemoryInBytes() == 481490)); + assertTrue("memory requirement is different", Iterables.all(totalResources.entrySet(), (e) -> e.getValue().getMemoryInBytes() == 481460)); } @Test diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/record/TestRecordIterator.java b/exec/java-exec/src/test/java/org/apache/drill/exec/record/TestRecordIterator.java index 01c06a4a664..37eb3556d14 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/record/TestRecordIterator.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/record/TestRecordIterator.java @@ -77,7 +77,7 @@ public void testSimpleIterator() throws Throwable { RecordBatch singleBatch = exec.getIncoming(); PhysicalOperator dummyPop = operatorList.iterator().next(); OpProfileDef def = new OpProfileDef(dummyPop.getOperatorId(), UserBitShared.CoreOperatorType.MOCK_SUB_SCAN_VALUE, - OperatorUtilities.getChildCount(dummyPop)); + OperatorUtilities.getChildCount(dummyPop), dummyPop.getMaxAllocation()); OperatorStats stats = exec.getContext().getStats().newOperatorStats(def, exec.getContext().getAllocator()); RecordIterator iter = new RecordIterator(singleBatch, null, exec.getContext().newOperatorContext(dummyPop, stats), 0, false, null); int totalRecords = 0; @@ -133,7 +133,7 @@ public void testMarkResetIterator() throws Throwable { RecordBatch singleBatch = exec.getIncoming(); PhysicalOperator dummyPop = operatorList.iterator().next(); OpProfileDef def = new OpProfileDef(dummyPop.getOperatorId(), UserBitShared.CoreOperatorType.MOCK_SUB_SCAN_VALUE, - OperatorUtilities.getChildCount(dummyPop)); + OperatorUtilities.getChildCount(dummyPop), dummyPop.getMaxAllocation()); OperatorStats stats = exec.getContext().getStats().newOperatorStats(def, exec.getContext().getAllocator()); RecordIterator iter = new RecordIterator(singleBatch, null, exec.getContext().newOperatorContext(dummyPop, stats), 0, null); List vectors; diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/dfs/TestDrillFileSystem.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/dfs/TestDrillFileSystem.java index 5504382dc09..b5ca7351d4e 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/dfs/TestDrillFileSystem.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/dfs/TestDrillFileSystem.java @@ -67,7 +67,7 @@ public void testIOStats() throws Exception { InputStream is = null; Configuration conf = new Configuration(); conf.set(FileSystem.FS_DEFAULT_NAME_KEY, FileSystem.DEFAULT_FS); - OpProfileDef profileDef = new OpProfileDef(0 /*operatorId*/, 0 /*operatorType*/, 0 /*inputCount*/); + OpProfileDef profileDef = new OpProfileDef(0 /*operatorId*/, 0 /*operatorType*/, 0 /*inputCount*/, 0 /*optimalMemoryAllocation*/); OperatorStats stats = new OperatorStats(profileDef, null /*allocator*/); // start wait time method in OperatorStats expects the OperatorStats state to be in "processing" diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/OperatorFixture.java b/exec/java-exec/src/test/java/org/apache/drill/test/OperatorFixture.java index 3d9190894ac..bc72bd8b781 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/test/OperatorFixture.java +++ b/exec/java-exec/src/test/java/org/apache/drill/test/OperatorFixture.java @@ -438,7 +438,7 @@ public MockOperatorContext(FragmentContext fragContext, BufferAllocator allocator, PhysicalOperator config) { super(fragContext, allocator, config); - this.operatorStats = new OperatorStats(new OpProfileDef(0, 0, 100), allocator); + this.operatorStats = new OperatorStats(new OpProfileDef(0, 0, 100, 0), allocator); } @Override diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/SchemaUserBitShared.java b/protocol/src/main/java/org/apache/drill/exec/proto/SchemaUserBitShared.java index 932872c5163..4babfb18148 100644 --- a/protocol/src/main/java/org/apache/drill/exec/proto/SchemaUserBitShared.java +++ b/protocol/src/main/java/org/apache/drill/exec/proto/SchemaUserBitShared.java @@ -2377,6 +2377,8 @@ public void writeTo(com.dyuproject.protostuff.Output output, org.apache.drill.ex if(message.hasWaitNanos()) output.writeInt64(9, message.getWaitNanos(), false); + if(message.hasOptimalMemAllocation()) + output.writeInt64(10, message.getOptimalMemAllocation(), false); } public boolean isInitialized(org.apache.drill.exec.proto.UserBitShared.OperatorProfile message) { @@ -2442,6 +2444,9 @@ public void mergeFrom(com.dyuproject.protostuff.Input input, org.apache.drill.ex case 9: builder.setWaitNanos(input.readInt64()); break; + case 10: + builder.setOptimalMemAllocation(input.readInt64()); + break; default: input.handleUnknownField(number, this); } @@ -2490,6 +2495,7 @@ public static java.lang.String getFieldName(int number) case 7: return "peakLocalMemoryAllocated"; case 8: return "metric"; case 9: return "waitNanos"; + case 10: return "optimalMemAllocation"; default: return null; } } @@ -2509,6 +2515,7 @@ public static int getFieldNumber(java.lang.String name) fieldMap.put("peakLocalMemoryAllocated", 7); fieldMap.put("metric", 8); fieldMap.put("waitNanos", 9); + fieldMap.put("optimalMemAllocation", 10); } } diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java b/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java index 5f30015ef4f..452d90e3bda 100644 --- a/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java +++ b/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java @@ -22363,6 +22363,15 @@ org.apache.drill.exec.proto.UserBitShared.MetricValueOrBuilder getMetricOrBuilde * optional int64 wait_nanos = 9; */ long getWaitNanos(); + + /** + * optional int64 optimal_mem_allocation = 10; + */ + boolean hasOptimalMemAllocation(); + /** + * optional int64 optimal_mem_allocation = 10; + */ + long getOptimalMemAllocation(); } /** * Protobuf type {@code exec.shared.OperatorProfile} @@ -22385,6 +22394,7 @@ private OperatorProfile() { peakLocalMemoryAllocated_ = 0L; metric_ = java.util.Collections.emptyList(); waitNanos_ = 0L; + optimalMemAllocation_ = 0L; } @java.lang.Override @@ -22459,6 +22469,11 @@ private OperatorProfile( waitNanos_ = input.readInt64(); break; } + case 80: { + bitField0_ |= 0x00000040; + optimalMemAllocation_ = input.readInt64(); + break; + } default: { if (!parseUnknownField( input, unknownFields, extensionRegistry, tag)) { @@ -22658,6 +22673,21 @@ public long getWaitNanos() { return waitNanos_; } + public static final int OPTIMAL_MEM_ALLOCATION_FIELD_NUMBER = 10; + private long optimalMemAllocation_; + /** + * optional int64 optimal_mem_allocation = 10; + */ + public boolean hasOptimalMemAllocation() { + return ((bitField0_ & 0x00000040) == 0x00000040); + } + /** + * optional int64 optimal_mem_allocation = 10; + */ + public long getOptimalMemAllocation() { + return optimalMemAllocation_; + } + private byte memoizedIsInitialized = -1; @java.lang.Override public final boolean isInitialized() { @@ -22696,6 +22726,9 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) if (((bitField0_ & 0x00000020) == 0x00000020)) { output.writeInt64(9, waitNanos_); } + if (((bitField0_ & 0x00000040) == 0x00000040)) { + output.writeInt64(10, optimalMemAllocation_); + } unknownFields.writeTo(output); } @@ -22737,6 +22770,10 @@ public int getSerializedSize() { size += com.google.protobuf.CodedOutputStream .computeInt64Size(9, waitNanos_); } + if (((bitField0_ & 0x00000040) == 0x00000040)) { + size += com.google.protobuf.CodedOutputStream + .computeInt64Size(10, optimalMemAllocation_); + } size += unknownFields.getSerializedSize(); memoizedSize = size; return size; @@ -22787,6 +22824,11 @@ public boolean equals(final java.lang.Object obj) { result = result && (getWaitNanos() == other.getWaitNanos()); } + result = result && (hasOptimalMemAllocation() == other.hasOptimalMemAllocation()); + if (hasOptimalMemAllocation()) { + result = result && (getOptimalMemAllocation() + == other.getOptimalMemAllocation()); + } result = result && unknownFields.equals(other.unknownFields); return result; } @@ -22834,6 +22876,11 @@ public int hashCode() { hash = (53 * hash) + com.google.protobuf.Internal.hashLong( getWaitNanos()); } + if (hasOptimalMemAllocation()) { + hash = (37 * hash) + OPTIMAL_MEM_ALLOCATION_FIELD_NUMBER; + hash = (53 * hash) + com.google.protobuf.Internal.hashLong( + getOptimalMemAllocation()); + } hash = (29 * hash) + unknownFields.hashCode(); memoizedHashCode = hash; return hash; @@ -22993,6 +23040,8 @@ public Builder clear() { } waitNanos_ = 0L; bitField0_ = (bitField0_ & ~0x00000080); + optimalMemAllocation_ = 0L; + bitField0_ = (bitField0_ & ~0x00000100); return this; } @@ -23063,6 +23112,10 @@ public org.apache.drill.exec.proto.UserBitShared.OperatorProfile buildPartial() to_bitField0_ |= 0x00000020; } result.waitNanos_ = waitNanos_; + if (((from_bitField0_ & 0x00000100) == 0x00000100)) { + to_bitField0_ |= 0x00000040; + } + result.optimalMemAllocation_ = optimalMemAllocation_; result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -23182,6 +23235,9 @@ public Builder mergeFrom(org.apache.drill.exec.proto.UserBitShared.OperatorProfi if (other.hasWaitNanos()) { setWaitNanos(other.getWaitNanos()); } + if (other.hasOptimalMemAllocation()) { + setOptimalMemAllocation(other.getOptimalMemAllocation()); + } this.mergeUnknownFields(other.unknownFields); onChanged(); return this; @@ -23883,6 +23939,38 @@ public Builder clearWaitNanos() { onChanged(); return this; } + + private long optimalMemAllocation_ ; + /** + * optional int64 optimal_mem_allocation = 10; + */ + public boolean hasOptimalMemAllocation() { + return ((bitField0_ & 0x00000100) == 0x00000100); + } + /** + * optional int64 optimal_mem_allocation = 10; + */ + public long getOptimalMemAllocation() { + return optimalMemAllocation_; + } + /** + * optional int64 optimal_mem_allocation = 10; + */ + public Builder setOptimalMemAllocation(long value) { + bitField0_ |= 0x00000100; + optimalMemAllocation_ = value; + onChanged(); + return this; + } + /** + * optional int64 optimal_mem_allocation = 10; + */ + public Builder clearOptimalMemAllocation() { + bitField0_ = (bitField0_ & ~0x00000100); + optimalMemAllocation_ = 0L; + onChanged(); + return this; + } @java.lang.Override public final Builder setUnknownFields( final com.google.protobuf.UnknownFieldSet unknownFields) { @@ -27858,64 +27946,65 @@ public org.apache.drill.exec.proto.UserBitShared.SaslMessage getDefaultInstanceF "y_used\030\007 \001(\003\022\027\n\017max_memory_used\030\010 \001(\003\022(\n" + "\010endpoint\030\t \001(\0132\026.exec.DrillbitEndpoint\022" + "\023\n\013last_update\030\n \001(\003\022\025\n\rlast_progress\030\013 " + - "\001(\003\"\377\001\n\017OperatorProfile\0221\n\rinput_profile" + + "\001(\003\"\237\002\n\017OperatorProfile\0221\n\rinput_profile" + "\030\001 \003(\0132\032.exec.shared.StreamProfile\022\023\n\013op" + "erator_id\030\003 \001(\005\022\025\n\roperator_type\030\004 \001(\005\022\023" + "\n\013setup_nanos\030\005 \001(\003\022\025\n\rprocess_nanos\030\006 \001" + "(\003\022#\n\033peak_local_memory_allocated\030\007 \001(\003\022" + "(\n\006metric\030\010 \003(\0132\030.exec.shared.MetricValu" + - "e\022\022\n\nwait_nanos\030\t \001(\003\"B\n\rStreamProfile\022\017" + - "\n\007records\030\001 \001(\003\022\017\n\007batches\030\002 \001(\003\022\017\n\007sche" + - "mas\030\003 \001(\003\"J\n\013MetricValue\022\021\n\tmetric_id\030\001 " + - "\001(\005\022\022\n\nlong_value\030\002 \001(\003\022\024\n\014double_value\030" + - "\003 \001(\001\")\n\010Registry\022\035\n\003jar\030\001 \003(\0132\020.exec.sh" + - "ared.Jar\"/\n\003Jar\022\014\n\004name\030\001 \001(\t\022\032\n\022functio" + - "n_signature\030\002 \003(\t\"W\n\013SaslMessage\022\021\n\tmech" + - "anism\030\001 \001(\t\022\014\n\004data\030\002 \001(\014\022\'\n\006status\030\003 \001(" + - "\0162\027.exec.shared.SaslStatus*5\n\nRpcChannel" + - "\022\017\n\013BIT_CONTROL\020\000\022\014\n\010BIT_DATA\020\001\022\010\n\004USER\020" + - "\002*V\n\tQueryType\022\007\n\003SQL\020\001\022\013\n\007LOGICAL\020\002\022\014\n\010" + - "PHYSICAL\020\003\022\r\n\tEXECUTION\020\004\022\026\n\022PREPARED_ST" + - "ATEMENT\020\005*\207\001\n\rFragmentState\022\013\n\007SENDING\020\000" + - "\022\027\n\023AWAITING_ALLOCATION\020\001\022\013\n\007RUNNING\020\002\022\014" + - "\n\010FINISHED\020\003\022\r\n\tCANCELLED\020\004\022\n\n\006FAILED\020\005\022" + - "\032\n\026CANCELLATION_REQUESTED\020\006*\374\t\n\020CoreOper" + - "atorType\022\021\n\rSINGLE_SENDER\020\000\022\024\n\020BROADCAST" + - "_SENDER\020\001\022\n\n\006FILTER\020\002\022\022\n\016HASH_AGGREGATE\020" + - "\003\022\r\n\tHASH_JOIN\020\004\022\016\n\nMERGE_JOIN\020\005\022\031\n\025HASH" + - "_PARTITION_SENDER\020\006\022\t\n\005LIMIT\020\007\022\024\n\020MERGIN" + - "G_RECEIVER\020\010\022\034\n\030ORDERED_PARTITION_SENDER" + - "\020\t\022\013\n\007PROJECT\020\n\022\026\n\022UNORDERED_RECEIVER\020\013\022" + - "\032\n\026RANGE_PARTITION_SENDER\020\014\022\n\n\006SCREEN\020\r\022" + - "\034\n\030SELECTION_VECTOR_REMOVER\020\016\022\027\n\023STREAMI" + - "NG_AGGREGATE\020\017\022\016\n\nTOP_N_SORT\020\020\022\021\n\rEXTERN" + - "AL_SORT\020\021\022\t\n\005TRACE\020\022\022\t\n\005UNION\020\023\022\014\n\010OLD_S" + - "ORT\020\024\022\032\n\026PARQUET_ROW_GROUP_SCAN\020\025\022\021\n\rHIV" + - "E_SUB_SCAN\020\026\022\025\n\021SYSTEM_TABLE_SCAN\020\027\022\021\n\rM" + - "OCK_SUB_SCAN\020\030\022\022\n\016PARQUET_WRITER\020\031\022\023\n\017DI" + - "RECT_SUB_SCAN\020\032\022\017\n\013TEXT_WRITER\020\033\022\021\n\rTEXT" + - "_SUB_SCAN\020\034\022\021\n\rJSON_SUB_SCAN\020\035\022\030\n\024INFO_S" + - "CHEMA_SUB_SCAN\020\036\022\023\n\017COMPLEX_TO_JSON\020\037\022\025\n" + - "\021PRODUCER_CONSUMER\020 \022\022\n\016HBASE_SUB_SCAN\020!" + - "\022\n\n\006WINDOW\020\"\022\024\n\020NESTED_LOOP_JOIN\020#\022\021\n\rAV" + - "RO_SUB_SCAN\020$\022\021\n\rPCAP_SUB_SCAN\020%\022\022\n\016KAFK" + - "A_SUB_SCAN\020&\022\021\n\rKUDU_SUB_SCAN\020\'\022\013\n\007FLATT" + - "EN\020(\022\020\n\014LATERAL_JOIN\020)\022\n\n\006UNNEST\020*\022,\n(HI" + - "VE_DRILL_NATIVE_PARQUET_ROW_GROUP_SCAN\020+" + - "\022\r\n\tJDBC_SCAN\020,\022\022\n\016REGEX_SUB_SCAN\020-\022\023\n\017M" + - "APRDB_SUB_SCAN\020.\022\022\n\016MONGO_SUB_SCAN\020/\022\017\n\013" + - "KUDU_WRITER\0200\022\026\n\022OPEN_TSDB_SUB_SCAN\0201\022\017\n" + - "\013JSON_WRITER\0202\022\026\n\022HTPPD_LOG_SUB_SCAN\0203\022\022" + - "\n\016IMAGE_SUB_SCAN\0204\022\025\n\021SEQUENCE_SUB_SCAN\020" + - "5\022\023\n\017PARTITION_LIMIT\0206\022\023\n\017PCAPNG_SUB_SCA" + - "N\0207\022\022\n\016RUNTIME_FILTER\0208\022\017\n\013ROWKEY_JOIN\0209" + - "\022\023\n\017SYSLOG_SUB_SCAN\020:\022\030\n\024STATISTICS_AGGR" + - "EGATE\020;\022\020\n\014UNPIVOT_MAPS\020<\022\024\n\020STATISTICS_" + - "MERGE\020=\022\021\n\rLTSV_SUB_SCAN\020>*g\n\nSaslStatus" + - "\022\020\n\014SASL_UNKNOWN\020\000\022\016\n\nSASL_START\020\001\022\024\n\020SA" + - "SL_IN_PROGRESS\020\002\022\020\n\014SASL_SUCCESS\020\003\022\017\n\013SA" + - "SL_FAILED\020\004B.\n\033org.apache.drill.exec.pro" + - "toB\rUserBitSharedH\001" + "e\022\022\n\nwait_nanos\030\t \001(\003\022\036\n\026optimal_mem_all" + + "ocation\030\n \001(\003\"B\n\rStreamProfile\022\017\n\007record" + + "s\030\001 \001(\003\022\017\n\007batches\030\002 \001(\003\022\017\n\007schemas\030\003 \001(" + + "\003\"J\n\013MetricValue\022\021\n\tmetric_id\030\001 \001(\005\022\022\n\nl" + + "ong_value\030\002 \001(\003\022\024\n\014double_value\030\003 \001(\001\")\n" + + "\010Registry\022\035\n\003jar\030\001 \003(\0132\020.exec.shared.Jar" + + "\"/\n\003Jar\022\014\n\004name\030\001 \001(\t\022\032\n\022function_signat" + + "ure\030\002 \003(\t\"W\n\013SaslMessage\022\021\n\tmechanism\030\001 " + + "\001(\t\022\014\n\004data\030\002 \001(\014\022\'\n\006status\030\003 \001(\0162\027.exec" + + ".shared.SaslStatus*5\n\nRpcChannel\022\017\n\013BIT_" + + "CONTROL\020\000\022\014\n\010BIT_DATA\020\001\022\010\n\004USER\020\002*V\n\tQue" + + "ryType\022\007\n\003SQL\020\001\022\013\n\007LOGICAL\020\002\022\014\n\010PHYSICAL" + + "\020\003\022\r\n\tEXECUTION\020\004\022\026\n\022PREPARED_STATEMENT\020" + + "\005*\207\001\n\rFragmentState\022\013\n\007SENDING\020\000\022\027\n\023AWAI" + + "TING_ALLOCATION\020\001\022\013\n\007RUNNING\020\002\022\014\n\010FINISH" + + "ED\020\003\022\r\n\tCANCELLED\020\004\022\n\n\006FAILED\020\005\022\032\n\026CANCE" + + "LLATION_REQUESTED\020\006*\374\t\n\020CoreOperatorType" + + "\022\021\n\rSINGLE_SENDER\020\000\022\024\n\020BROADCAST_SENDER\020" + + "\001\022\n\n\006FILTER\020\002\022\022\n\016HASH_AGGREGATE\020\003\022\r\n\tHAS" + + "H_JOIN\020\004\022\016\n\nMERGE_JOIN\020\005\022\031\n\025HASH_PARTITI" + + "ON_SENDER\020\006\022\t\n\005LIMIT\020\007\022\024\n\020MERGING_RECEIV" + + "ER\020\010\022\034\n\030ORDERED_PARTITION_SENDER\020\t\022\013\n\007PR" + + "OJECT\020\n\022\026\n\022UNORDERED_RECEIVER\020\013\022\032\n\026RANGE" + + "_PARTITION_SENDER\020\014\022\n\n\006SCREEN\020\r\022\034\n\030SELEC" + + "TION_VECTOR_REMOVER\020\016\022\027\n\023STREAMING_AGGRE" + + "GATE\020\017\022\016\n\nTOP_N_SORT\020\020\022\021\n\rEXTERNAL_SORT\020" + + "\021\022\t\n\005TRACE\020\022\022\t\n\005UNION\020\023\022\014\n\010OLD_SORT\020\024\022\032\n" + + "\026PARQUET_ROW_GROUP_SCAN\020\025\022\021\n\rHIVE_SUB_SC" + + "AN\020\026\022\025\n\021SYSTEM_TABLE_SCAN\020\027\022\021\n\rMOCK_SUB_" + + "SCAN\020\030\022\022\n\016PARQUET_WRITER\020\031\022\023\n\017DIRECT_SUB" + + "_SCAN\020\032\022\017\n\013TEXT_WRITER\020\033\022\021\n\rTEXT_SUB_SCA" + + "N\020\034\022\021\n\rJSON_SUB_SCAN\020\035\022\030\n\024INFO_SCHEMA_SU" + + "B_SCAN\020\036\022\023\n\017COMPLEX_TO_JSON\020\037\022\025\n\021PRODUCE" + + "R_CONSUMER\020 \022\022\n\016HBASE_SUB_SCAN\020!\022\n\n\006WIND" + + "OW\020\"\022\024\n\020NESTED_LOOP_JOIN\020#\022\021\n\rAVRO_SUB_S" + + "CAN\020$\022\021\n\rPCAP_SUB_SCAN\020%\022\022\n\016KAFKA_SUB_SC" + + "AN\020&\022\021\n\rKUDU_SUB_SCAN\020\'\022\013\n\007FLATTEN\020(\022\020\n\014" + + "LATERAL_JOIN\020)\022\n\n\006UNNEST\020*\022,\n(HIVE_DRILL" + + "_NATIVE_PARQUET_ROW_GROUP_SCAN\020+\022\r\n\tJDBC" + + "_SCAN\020,\022\022\n\016REGEX_SUB_SCAN\020-\022\023\n\017MAPRDB_SU" + + "B_SCAN\020.\022\022\n\016MONGO_SUB_SCAN\020/\022\017\n\013KUDU_WRI" + + "TER\0200\022\026\n\022OPEN_TSDB_SUB_SCAN\0201\022\017\n\013JSON_WR" + + "ITER\0202\022\026\n\022HTPPD_LOG_SUB_SCAN\0203\022\022\n\016IMAGE_" + + "SUB_SCAN\0204\022\025\n\021SEQUENCE_SUB_SCAN\0205\022\023\n\017PAR" + + "TITION_LIMIT\0206\022\023\n\017PCAPNG_SUB_SCAN\0207\022\022\n\016R" + + "UNTIME_FILTER\0208\022\017\n\013ROWKEY_JOIN\0209\022\023\n\017SYSL" + + "OG_SUB_SCAN\020:\022\030\n\024STATISTICS_AGGREGATE\020;\022" + + "\020\n\014UNPIVOT_MAPS\020<\022\024\n\020STATISTICS_MERGE\020=\022" + + "\021\n\rLTSV_SUB_SCAN\020>*g\n\nSaslStatus\022\020\n\014SASL" + + "_UNKNOWN\020\000\022\016\n\nSASL_START\020\001\022\024\n\020SASL_IN_PR" + + "OGRESS\020\002\022\020\n\014SASL_SUCCESS\020\003\022\017\n\013SASL_FAILE" + + "D\020\004B.\n\033org.apache.drill.exec.protoB\rUser" + + "BitSharedH\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor. InternalDescriptorAssigner() { @@ -28033,7 +28122,7 @@ public com.google.protobuf.ExtensionRegistry assignDescriptors( internal_static_exec_shared_OperatorProfile_fieldAccessorTable = new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( internal_static_exec_shared_OperatorProfile_descriptor, - new java.lang.String[] { "InputProfile", "OperatorId", "OperatorType", "SetupNanos", "ProcessNanos", "PeakLocalMemoryAllocated", "Metric", "WaitNanos", }); + new java.lang.String[] { "InputProfile", "OperatorId", "OperatorType", "SetupNanos", "ProcessNanos", "PeakLocalMemoryAllocated", "Metric", "WaitNanos", "OptimalMemAllocation", }); internal_static_exec_shared_StreamProfile_descriptor = getDescriptor().getMessageTypes().get(17); internal_static_exec_shared_StreamProfile_fieldAccessorTable = new diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/beans/OperatorProfile.java b/protocol/src/main/java/org/apache/drill/exec/proto/beans/OperatorProfile.java index d6275fa26c3..224214a69dd 100644 --- a/protocol/src/main/java/org/apache/drill/exec/proto/beans/OperatorProfile.java +++ b/protocol/src/main/java/org/apache/drill/exec/proto/beans/OperatorProfile.java @@ -57,6 +57,7 @@ public static OperatorProfile getDefaultInstance() private long peakLocalMemoryAllocated; private List metric; private long waitNanos; + private long optimalMemAllocation; public OperatorProfile() { @@ -169,6 +170,19 @@ public OperatorProfile setWaitNanos(long waitNanos) return this; } + // optimalMemAllocation + + public long getOptimalMemAllocation() + { + return optimalMemAllocation; + } + + public OperatorProfile setOptimalMemAllocation(long optimalMemAllocation) + { + this.optimalMemAllocation = optimalMemAllocation; + return this; + } + // java serialization public void readExternal(ObjectInput in) throws IOException @@ -253,6 +267,9 @@ public void mergeFrom(Input input, OperatorProfile message) throws IOException case 9: message.waitNanos = input.readInt64(); break; + case 10: + message.optimalMemAllocation = input.readInt64(); + break; default: input.handleUnknownField(number, this); } @@ -299,6 +316,9 @@ public void writeTo(Output output, OperatorProfile message) throws IOException if(message.waitNanos != 0) output.writeInt64(9, message.waitNanos, false); + + if(message.optimalMemAllocation != 0) + output.writeInt64(10, message.optimalMemAllocation, false); } public String getFieldName(int number) @@ -313,6 +333,7 @@ public String getFieldName(int number) case 7: return "peakLocalMemoryAllocated"; case 8: return "metric"; case 9: return "waitNanos"; + case 10: return "optimalMemAllocation"; default: return null; } } @@ -334,6 +355,7 @@ public int getFieldNumber(String name) __fieldMap.put("peakLocalMemoryAllocated", 7); __fieldMap.put("metric", 8); __fieldMap.put("waitNanos", 9); + __fieldMap.put("optimalMemAllocation", 10); } } diff --git a/protocol/src/main/protobuf/UserBitShared.proto b/protocol/src/main/protobuf/UserBitShared.proto index 4d057d59e3a..46d28eb4146 100644 --- a/protocol/src/main/protobuf/UserBitShared.proto +++ b/protocol/src/main/protobuf/UserBitShared.proto @@ -268,6 +268,7 @@ message OperatorProfile { optional int64 peak_local_memory_allocated = 7; repeated MetricValue metric = 8; optional int64 wait_nanos = 9; + optional int64 optimal_mem_allocation = 10; } message StreamProfile { From a3f8b3669bcb771ecb25de50d6d7f1431e763d8d Mon Sep 17 00:00:00 2001 From: HanumathRao Date: Wed, 8 May 2019 12:04:19 -0700 Subject: [PATCH 7/7] Addressing Review comments. --- .../client/src/protobuf/BitControl.pb.cc | 104 +++++----- .../client/src/protobuf/BitControl.pb.h | 112 +++++------ .../client/src/protobuf/UserBitShared.pb.cc | 150 +++++++------- .../client/src/protobuf/UserBitShared.pb.h | 46 ++--- .../org/apache/drill/common/DrillNode.java | 43 ++-- .../apache/drill/exec/ops/OperatorStats.java | 6 +- .../DistributedQueueParallelizer.java | 20 +- .../drill/exec/planner/fragment/Fragment.java | 20 +- .../planner/fragment/MemoryCalculator.java | 6 +- .../planner/fragment/SimpleParallelizer.java | 2 +- .../planner/fragment/ZKQueueParallelizer.java | 20 +- .../drill/exec/resourcemgr/NodeResources.java | 12 +- .../DefaultMemoryAllocationUtilities.java | 84 +------- .../memory/MemoryAllocationUtilities.java | 113 +++++++++++ .../ZKQueueMemoryAllocationUtilities.java | 130 +----------- .../drill/exec/work/user/PlanSplitter.java | 1 - .../apache/drill/exec/proto/BitControl.java | 158 +++++++-------- .../drill/exec/proto/SchemaBitControl.java | 10 +- .../drill/exec/proto/SchemaUserBitShared.java | 10 +- .../drill/exec/proto/UserBitShared.java | 186 +++++++++--------- .../exec/proto/beans/OperatorProfile.java | 22 +-- .../drill/exec/proto/beans/PlanFragment.java | 22 +-- protocol/src/main/protobuf/BitControl.proto | 2 +- .../src/main/protobuf/UserBitShared.proto | 2 +- 24 files changed, 595 insertions(+), 686 deletions(-) create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/util/memory/MemoryAllocationUtilities.java diff --git a/contrib/native/client/src/protobuf/BitControl.pb.cc b/contrib/native/client/src/protobuf/BitControl.pb.cc index e31c7265a31..a3e197d505c 100644 --- a/contrib/native/client/src/protobuf/BitControl.pb.cc +++ b/contrib/native/client/src/protobuf/BitControl.pb.cc @@ -326,7 +326,7 @@ const ::google::protobuf::uint32 TableStruct::offsets[] GOOGLE_PROTOBUF_ATTRIBUT GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(::exec::bit::control::PlanFragment, options_json_), GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(::exec::bit::control::PlanFragment, context_), GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(::exec::bit::control::PlanFragment, collector_), - GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(::exec::bit::control::PlanFragment, endpointuuid_), + GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(::exec::bit::control::PlanFragment, assignedendpointuuid_), 3, 8, 9, @@ -451,7 +451,7 @@ void AddDescriptorsImpl() { "c.bit.FragmentHandle\"G\n\023InitializeFragme" "nts\0220\n\010fragment\030\001 \003(\0132\036.exec.bit.control" ".PlanFragment\".\n\rCustomMessage\022\014\n\004type\030\001" - " \001(\005\022\017\n\007message\030\002 \001(\014\"\222\004\n\014PlanFragment\022(" + " \001(\005\022\017\n\007message\030\002 \001(\014\"\232\004\n\014PlanFragment\022(" "\n\006handle\030\001 \001(\0132\030.exec.bit.FragmentHandle" "\022\024\n\014network_cost\030\004 \001(\002\022\020\n\010cpu_cost\030\005 \001(\002" "\022\021\n\tdisk_cost\030\006 \001(\002\022\023\n\013memory_cost\030\007 \001(\002" @@ -464,33 +464,33 @@ void AddDescriptorsImpl() { "ls\022\024\n\014options_json\030\017 \001(\t\022:\n\007context\030\020 \001(" "\0132).exec.bit.control.QueryContextInforma" "tion\022.\n\tcollector\030\021 \003(\0132\033.exec.bit.contr" - "ol.Collector\022\024\n\014endpointUUID\030\022 \001(\t\"\210\001\n\tC" - "ollector\022\"\n\032opposite_major_fragment_id\030\001" - " \001(\005\022#\n\027incoming_minor_fragment\030\002 \003(\005B\002\020" - "\001\022\035\n\025supports_out_of_order\030\003 \001(\010\022\023\n\013is_s" - "pooling\030\004 \001(\010\"w\n\027QueryContextInformation" - "\022\030\n\020query_start_time\030\001 \001(\003\022\021\n\ttime_zone\030" - "\002 \001(\005\022\033\n\023default_schema_name\030\003 \001(\t\022\022\n\nse" - "ssion_id\030\004 \001(\t\"f\n\017WorkQueueStatus\022(\n\010end" - "point\030\001 \001(\0132\026.exec.DrillbitEndpoint\022\024\n\014q" - "ueue_length\030\002 \001(\005\022\023\n\013report_time\030\003 \001(\003\"h" - "\n\020FinishedReceiver\022*\n\010receiver\030\001 \001(\0132\030.e" - "xec.bit.FragmentHandle\022(\n\006sender\030\002 \001(\0132\030" - ".exec.bit.FragmentHandle*\206\003\n\007RpcType\022\r\n\t" - "HANDSHAKE\020\000\022\007\n\003ACK\020\001\022\013\n\007GOODBYE\020\002\022\034\n\030REQ" - "_INITIALIZE_FRAGMENTS\020\003\022\027\n\023REQ_CANCEL_FR" - "AGMENT\020\006\022\031\n\025REQ_RECEIVER_FINISHED\020\007\022\027\n\023R" - "EQ_FRAGMENT_STATUS\020\010\022\022\n\016REQ_BIT_STATUS\020\t" - "\022\024\n\020REQ_QUERY_STATUS\020\n\022\024\n\020REQ_QUERY_CANC" - "EL\020\017\022\030\n\024REQ_UNPAUSE_FRAGMENT\020\020\022\016\n\nREQ_CU" - "STOM\020\021\022\030\n\024RESP_FRAGMENT_HANDLE\020\013\022\030\n\024RESP" - "_FRAGMENT_STATUS\020\014\022\023\n\017RESP_BIT_STATUS\020\r\022" - "\025\n\021RESP_QUERY_STATUS\020\016\022\017\n\013RESP_CUSTOM\020\022\022" - "\020\n\014SASL_MESSAGE\020\023B+\n\033org.apache.drill.ex" - "ec.protoB\nBitControlH\001" + "ol.Collector\022\034\n\024assignedEndpointUUID\030\022 \001" + "(\t\"\210\001\n\tCollector\022\"\n\032opposite_major_fragm" + "ent_id\030\001 \001(\005\022#\n\027incoming_minor_fragment\030" + "\002 \003(\005B\002\020\001\022\035\n\025supports_out_of_order\030\003 \001(\010" + "\022\023\n\013is_spooling\030\004 \001(\010\"w\n\027QueryContextInf" + "ormation\022\030\n\020query_start_time\030\001 \001(\003\022\021\n\tti" + "me_zone\030\002 \001(\005\022\033\n\023default_schema_name\030\003 \001" + "(\t\022\022\n\nsession_id\030\004 \001(\t\"f\n\017WorkQueueStatu" + "s\022(\n\010endpoint\030\001 \001(\0132\026.exec.DrillbitEndpo" + "int\022\024\n\014queue_length\030\002 \001(\005\022\023\n\013report_time" + "\030\003 \001(\003\"h\n\020FinishedReceiver\022*\n\010receiver\030\001" + " \001(\0132\030.exec.bit.FragmentHandle\022(\n\006sender" + "\030\002 \001(\0132\030.exec.bit.FragmentHandle*\206\003\n\007Rpc" + "Type\022\r\n\tHANDSHAKE\020\000\022\007\n\003ACK\020\001\022\013\n\007GOODBYE\020" + "\002\022\034\n\030REQ_INITIALIZE_FRAGMENTS\020\003\022\027\n\023REQ_C" + "ANCEL_FRAGMENT\020\006\022\031\n\025REQ_RECEIVER_FINISHE" + "D\020\007\022\027\n\023REQ_FRAGMENT_STATUS\020\010\022\022\n\016REQ_BIT_" + "STATUS\020\t\022\024\n\020REQ_QUERY_STATUS\020\n\022\024\n\020REQ_QU" + "ERY_CANCEL\020\017\022\030\n\024REQ_UNPAUSE_FRAGMENT\020\020\022\016" + "\n\nREQ_CUSTOM\020\021\022\030\n\024RESP_FRAGMENT_HANDLE\020\013" + "\022\030\n\024RESP_FRAGMENT_STATUS\020\014\022\023\n\017RESP_BIT_S" + "TATUS\020\r\022\025\n\021RESP_QUERY_STATUS\020\016\022\017\n\013RESP_C" + "USTOM\020\022\022\020\n\014SASL_MESSAGE\020\023B+\n\033org.apache." + "drill.exec.protoB\nBitControlH\001" }; ::google::protobuf::DescriptorPool::InternalAddGeneratedFile( - descriptor, 2022); + descriptor, 2030); ::google::protobuf::MessageFactory::InternalRegisterGeneratedFile( "BitControl.proto", &protobuf_RegisterTypes); ::protobuf_ExecutionProtos_2eproto::AddDescriptors(); @@ -2031,7 +2031,7 @@ const int PlanFragment::kCredentialsFieldNumber; const int PlanFragment::kOptionsJsonFieldNumber; const int PlanFragment::kContextFieldNumber; const int PlanFragment::kCollectorFieldNumber; -const int PlanFragment::kEndpointUUIDFieldNumber; +const int PlanFragment::kAssignedEndpointUUIDFieldNumber; #endif // !defined(_MSC_VER) || _MSC_VER >= 1900 PlanFragment::PlanFragment() @@ -2055,9 +2055,9 @@ PlanFragment::PlanFragment(const PlanFragment& from) if (from.has_options_json()) { options_json_.AssignWithDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), from.options_json_); } - endpointuuid_.UnsafeSetDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - if (from.has_endpointuuid()) { - endpointuuid_.AssignWithDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), from.endpointuuid_); + assignedendpointuuid_.UnsafeSetDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); + if (from.has_assignedendpointuuid()) { + assignedendpointuuid_.AssignWithDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), from.assignedendpointuuid_); } if (from.has_handle()) { handle_ = new ::exec::bit::FragmentHandle(*from.handle_); @@ -2093,7 +2093,7 @@ PlanFragment::PlanFragment(const PlanFragment& from) void PlanFragment::SharedCtor() { fragment_json_.UnsafeSetDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); options_json_.UnsafeSetDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - endpointuuid_.UnsafeSetDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); + assignedendpointuuid_.UnsafeSetDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); ::memset(&handle_, 0, static_cast( reinterpret_cast(&leaf_fragment_) - reinterpret_cast(&handle_)) + sizeof(leaf_fragment_)); @@ -2109,7 +2109,7 @@ PlanFragment::~PlanFragment() { void PlanFragment::SharedDtor() { fragment_json_.DestroyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); options_json_.DestroyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - endpointuuid_.DestroyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); + assignedendpointuuid_.DestroyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); if (this != internal_default_instance()) delete handle_; if (this != internal_default_instance()) delete assignment_; if (this != internal_default_instance()) delete foreman_; @@ -2147,7 +2147,7 @@ void PlanFragment::Clear() { options_json_.ClearNonDefaultToEmptyNoArena(); } if (cached_has_bits & 0x00000004u) { - endpointuuid_.ClearNonDefaultToEmptyNoArena(); + assignedendpointuuid_.ClearNonDefaultToEmptyNoArena(); } if (cached_has_bits & 0x00000008u) { GOOGLE_DCHECK(handle_ != NULL); @@ -2393,16 +2393,16 @@ bool PlanFragment::MergePartialFromCodedStream( break; } - // optional string endpointUUID = 18; + // optional string assignedEndpointUUID = 18; case 18: { if (static_cast< ::google::protobuf::uint8>(tag) == static_cast< ::google::protobuf::uint8>(146u /* 146 & 0xFF */)) { DO_(::google::protobuf::internal::WireFormatLite::ReadString( - input, this->mutable_endpointuuid())); + input, this->mutable_assignedendpointuuid())); ::google::protobuf::internal::WireFormat::VerifyUTF8StringNamedField( - this->endpointuuid().data(), static_cast(this->endpointuuid().length()), + this->assignedendpointuuid().data(), static_cast(this->assignedendpointuuid().length()), ::google::protobuf::internal::WireFormat::PARSE, - "exec.bit.control.PlanFragment.endpointUUID"); + "exec.bit.control.PlanFragment.assignedEndpointUUID"); } else { goto handle_unusual; } @@ -2530,14 +2530,14 @@ void PlanFragment::SerializeWithCachedSizes( output); } - // optional string endpointUUID = 18; + // optional string assignedEndpointUUID = 18; if (cached_has_bits & 0x00000004u) { ::google::protobuf::internal::WireFormat::VerifyUTF8StringNamedField( - this->endpointuuid().data(), static_cast(this->endpointuuid().length()), + this->assignedendpointuuid().data(), static_cast(this->assignedendpointuuid().length()), ::google::protobuf::internal::WireFormat::SERIALIZE, - "exec.bit.control.PlanFragment.endpointUUID"); + "exec.bit.control.PlanFragment.assignedEndpointUUID"); ::google::protobuf::internal::WireFormatLite::WriteStringMaybeAliased( - 18, this->endpointuuid(), output); + 18, this->assignedendpointuuid(), output); } if (_internal_metadata_.have_unknown_fields()) { @@ -2655,15 +2655,15 @@ ::google::protobuf::uint8* PlanFragment::InternalSerializeWithCachedSizesToArray 17, this->collector(static_cast(i)), deterministic, target); } - // optional string endpointUUID = 18; + // optional string assignedEndpointUUID = 18; if (cached_has_bits & 0x00000004u) { ::google::protobuf::internal::WireFormat::VerifyUTF8StringNamedField( - this->endpointuuid().data(), static_cast(this->endpointuuid().length()), + this->assignedendpointuuid().data(), static_cast(this->assignedendpointuuid().length()), ::google::protobuf::internal::WireFormat::SERIALIZE, - "exec.bit.control.PlanFragment.endpointUUID"); + "exec.bit.control.PlanFragment.assignedEndpointUUID"); target = ::google::protobuf::internal::WireFormatLite::WriteStringToArray( - 18, this->endpointuuid(), target); + 18, this->assignedendpointuuid(), target); } if (_internal_metadata_.have_unknown_fields()) { @@ -2709,11 +2709,11 @@ size_t PlanFragment::ByteSizeLong() const { this->options_json()); } - // optional string endpointUUID = 18; - if (has_endpointuuid()) { + // optional string assignedEndpointUUID = 18; + if (has_assignedendpointuuid()) { total_size += 2 + ::google::protobuf::internal::WireFormatLite::StringSize( - this->endpointuuid()); + this->assignedendpointuuid()); } // optional .exec.bit.FragmentHandle handle = 1; @@ -2832,8 +2832,8 @@ void PlanFragment::MergeFrom(const PlanFragment& from) { options_json_.AssignWithDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), from.options_json_); } if (cached_has_bits & 0x00000004u) { - set_has_endpointuuid(); - endpointuuid_.AssignWithDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), from.endpointuuid_); + set_has_assignedendpointuuid(); + assignedendpointuuid_.AssignWithDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), from.assignedendpointuuid_); } if (cached_has_bits & 0x00000008u) { mutable_handle()->::exec::bit::FragmentHandle::MergeFrom(from.handle()); @@ -2906,7 +2906,7 @@ void PlanFragment::InternalSwap(PlanFragment* other) { GetArenaNoVirtual()); options_json_.Swap(&other->options_json_, &::google::protobuf::internal::GetEmptyStringAlreadyInited(), GetArenaNoVirtual()); - endpointuuid_.Swap(&other->endpointuuid_, &::google::protobuf::internal::GetEmptyStringAlreadyInited(), + assignedendpointuuid_.Swap(&other->assignedendpointuuid_, &::google::protobuf::internal::GetEmptyStringAlreadyInited(), GetArenaNoVirtual()); swap(handle_, other->handle_); swap(assignment_, other->assignment_); diff --git a/contrib/native/client/src/protobuf/BitControl.pb.h b/contrib/native/client/src/protobuf/BitControl.pb.h index 58e4889946f..44cc604b3f4 100644 --- a/contrib/native/client/src/protobuf/BitControl.pb.h +++ b/contrib/native/client/src/protobuf/BitControl.pb.h @@ -938,20 +938,20 @@ class PlanFragment : public ::google::protobuf::Message /* @@protoc_insertion_po ::std::string* release_options_json(); void set_allocated_options_json(::std::string* options_json); - // optional string endpointUUID = 18; - bool has_endpointuuid() const; - void clear_endpointuuid(); - static const int kEndpointUUIDFieldNumber = 18; - const ::std::string& endpointuuid() const; - void set_endpointuuid(const ::std::string& value); + // optional string assignedEndpointUUID = 18; + bool has_assignedendpointuuid() const; + void clear_assignedendpointuuid(); + static const int kAssignedEndpointUUIDFieldNumber = 18; + const ::std::string& assignedendpointuuid() const; + void set_assignedendpointuuid(const ::std::string& value); #if LANG_CXX11 - void set_endpointuuid(::std::string&& value); + void set_assignedendpointuuid(::std::string&& value); #endif - void set_endpointuuid(const char* value); - void set_endpointuuid(const char* value, size_t size); - ::std::string* mutable_endpointuuid(); - ::std::string* release_endpointuuid(); - void set_allocated_endpointuuid(::std::string* endpointuuid); + void set_assignedendpointuuid(const char* value); + void set_assignedendpointuuid(const char* value, size_t size); + ::std::string* mutable_assignedendpointuuid(); + ::std::string* release_assignedendpointuuid(); + void set_allocated_assignedendpointuuid(::std::string* assignedendpointuuid); // optional .exec.bit.FragmentHandle handle = 1; bool has_handle() const; @@ -1092,8 +1092,8 @@ class PlanFragment : public ::google::protobuf::Message /* @@protoc_insertion_po void clear_has_options_json(); void set_has_context(); void clear_has_context(); - void set_has_endpointuuid(); - void clear_has_endpointuuid(); + void set_has_assignedendpointuuid(); + void clear_has_assignedendpointuuid(); ::google::protobuf::internal::InternalMetadataWithArena _internal_metadata_; ::google::protobuf::internal::HasBits<1> _has_bits_; @@ -1101,7 +1101,7 @@ class PlanFragment : public ::google::protobuf::Message /* @@protoc_insertion_po ::google::protobuf::RepeatedPtrField< ::exec::bit::control::Collector > collector_; ::google::protobuf::internal::ArenaStringPtr fragment_json_; ::google::protobuf::internal::ArenaStringPtr options_json_; - ::google::protobuf::internal::ArenaStringPtr endpointuuid_; + ::google::protobuf::internal::ArenaStringPtr assignedendpointuuid_; ::exec::bit::FragmentHandle* handle_; ::exec::DrillbitEndpoint* assignment_; ::exec::DrillbitEndpoint* foreman_; @@ -2762,70 +2762,70 @@ PlanFragment::collector() const { return collector_; } -// optional string endpointUUID = 18; -inline bool PlanFragment::has_endpointuuid() const { +// optional string assignedEndpointUUID = 18; +inline bool PlanFragment::has_assignedendpointuuid() const { return (_has_bits_[0] & 0x00000004u) != 0; } -inline void PlanFragment::set_has_endpointuuid() { +inline void PlanFragment::set_has_assignedendpointuuid() { _has_bits_[0] |= 0x00000004u; } -inline void PlanFragment::clear_has_endpointuuid() { +inline void PlanFragment::clear_has_assignedendpointuuid() { _has_bits_[0] &= ~0x00000004u; } -inline void PlanFragment::clear_endpointuuid() { - endpointuuid_.ClearToEmptyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); - clear_has_endpointuuid(); +inline void PlanFragment::clear_assignedendpointuuid() { + assignedendpointuuid_.ClearToEmptyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); + clear_has_assignedendpointuuid(); } -inline const ::std::string& PlanFragment::endpointuuid() const { - // @@protoc_insertion_point(field_get:exec.bit.control.PlanFragment.endpointUUID) - return endpointuuid_.GetNoArena(); +inline const ::std::string& PlanFragment::assignedendpointuuid() const { + // @@protoc_insertion_point(field_get:exec.bit.control.PlanFragment.assignedEndpointUUID) + return assignedendpointuuid_.GetNoArena(); } -inline void PlanFragment::set_endpointuuid(const ::std::string& value) { - set_has_endpointuuid(); - endpointuuid_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), value); - // @@protoc_insertion_point(field_set:exec.bit.control.PlanFragment.endpointUUID) +inline void PlanFragment::set_assignedendpointuuid(const ::std::string& value) { + set_has_assignedendpointuuid(); + assignedendpointuuid_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), value); + // @@protoc_insertion_point(field_set:exec.bit.control.PlanFragment.assignedEndpointUUID) } #if LANG_CXX11 -inline void PlanFragment::set_endpointuuid(::std::string&& value) { - set_has_endpointuuid(); - endpointuuid_.SetNoArena( +inline void PlanFragment::set_assignedendpointuuid(::std::string&& value) { + set_has_assignedendpointuuid(); + assignedendpointuuid_.SetNoArena( &::google::protobuf::internal::GetEmptyStringAlreadyInited(), ::std::move(value)); - // @@protoc_insertion_point(field_set_rvalue:exec.bit.control.PlanFragment.endpointUUID) + // @@protoc_insertion_point(field_set_rvalue:exec.bit.control.PlanFragment.assignedEndpointUUID) } #endif -inline void PlanFragment::set_endpointuuid(const char* value) { +inline void PlanFragment::set_assignedendpointuuid(const char* value) { GOOGLE_DCHECK(value != NULL); - set_has_endpointuuid(); - endpointuuid_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), ::std::string(value)); - // @@protoc_insertion_point(field_set_char:exec.bit.control.PlanFragment.endpointUUID) + set_has_assignedendpointuuid(); + assignedendpointuuid_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), ::std::string(value)); + // @@protoc_insertion_point(field_set_char:exec.bit.control.PlanFragment.assignedEndpointUUID) } -inline void PlanFragment::set_endpointuuid(const char* value, size_t size) { - set_has_endpointuuid(); - endpointuuid_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), +inline void PlanFragment::set_assignedendpointuuid(const char* value, size_t size) { + set_has_assignedendpointuuid(); + assignedendpointuuid_.SetNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), ::std::string(reinterpret_cast(value), size)); - // @@protoc_insertion_point(field_set_pointer:exec.bit.control.PlanFragment.endpointUUID) + // @@protoc_insertion_point(field_set_pointer:exec.bit.control.PlanFragment.assignedEndpointUUID) } -inline ::std::string* PlanFragment::mutable_endpointuuid() { - set_has_endpointuuid(); - // @@protoc_insertion_point(field_mutable:exec.bit.control.PlanFragment.endpointUUID) - return endpointuuid_.MutableNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); +inline ::std::string* PlanFragment::mutable_assignedendpointuuid() { + set_has_assignedendpointuuid(); + // @@protoc_insertion_point(field_mutable:exec.bit.control.PlanFragment.assignedEndpointUUID) + return assignedendpointuuid_.MutableNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); } -inline ::std::string* PlanFragment::release_endpointuuid() { - // @@protoc_insertion_point(field_release:exec.bit.control.PlanFragment.endpointUUID) - if (!has_endpointuuid()) { +inline ::std::string* PlanFragment::release_assignedendpointuuid() { + // @@protoc_insertion_point(field_release:exec.bit.control.PlanFragment.assignedEndpointUUID) + if (!has_assignedendpointuuid()) { return NULL; } - clear_has_endpointuuid(); - return endpointuuid_.ReleaseNonDefaultNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); + clear_has_assignedendpointuuid(); + return assignedendpointuuid_.ReleaseNonDefaultNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); } -inline void PlanFragment::set_allocated_endpointuuid(::std::string* endpointuuid) { - if (endpointuuid != NULL) { - set_has_endpointuuid(); +inline void PlanFragment::set_allocated_assignedendpointuuid(::std::string* assignedendpointuuid) { + if (assignedendpointuuid != NULL) { + set_has_assignedendpointuuid(); } else { - clear_has_endpointuuid(); + clear_has_assignedendpointuuid(); } - endpointuuid_.SetAllocatedNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), endpointuuid); - // @@protoc_insertion_point(field_set_allocated:exec.bit.control.PlanFragment.endpointUUID) + assignedendpointuuid_.SetAllocatedNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), assignedendpointuuid); + // @@protoc_insertion_point(field_set_allocated:exec.bit.control.PlanFragment.assignedEndpointUUID) } // ------------------------------------------------------------------- diff --git a/contrib/native/client/src/protobuf/UserBitShared.pb.cc b/contrib/native/client/src/protobuf/UserBitShared.pb.cc index 80e728a8834..71bc383bf0e 100644 --- a/contrib/native/client/src/protobuf/UserBitShared.pb.cc +++ b/contrib/native/client/src/protobuf/UserBitShared.pb.cc @@ -800,7 +800,7 @@ const ::google::protobuf::uint32 TableStruct::offsets[] GOOGLE_PROTOBUF_ATTRIBUT GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(::exec::shared::OperatorProfile, peak_local_memory_allocated_), GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(::exec::shared::OperatorProfile, metric_), GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(::exec::shared::OperatorProfile, wait_nanos_), - GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(::exec::shared::OperatorProfile, optimal_mem_allocation_), + GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(::exec::shared::OperatorProfile, max_allocation_), ~0u, 0, 1, @@ -1015,68 +1015,68 @@ void AddDescriptorsImpl() { "y_used\030\007 \001(\003\022\027\n\017max_memory_used\030\010 \001(\003\022(\n" "\010endpoint\030\t \001(\0132\026.exec.DrillbitEndpoint\022" "\023\n\013last_update\030\n \001(\003\022\025\n\rlast_progress\030\013 " - "\001(\003\"\237\002\n\017OperatorProfile\0221\n\rinput_profile" + "\001(\003\"\227\002\n\017OperatorProfile\0221\n\rinput_profile" "\030\001 \003(\0132\032.exec.shared.StreamProfile\022\023\n\013op" "erator_id\030\003 \001(\005\022\025\n\roperator_type\030\004 \001(\005\022\023" "\n\013setup_nanos\030\005 \001(\003\022\025\n\rprocess_nanos\030\006 \001" "(\003\022#\n\033peak_local_memory_allocated\030\007 \001(\003\022" "(\n\006metric\030\010 \003(\0132\030.exec.shared.MetricValu" - "e\022\022\n\nwait_nanos\030\t \001(\003\022\036\n\026optimal_mem_all" - "ocation\030\n \001(\003\"B\n\rStreamProfile\022\017\n\007record" - "s\030\001 \001(\003\022\017\n\007batches\030\002 \001(\003\022\017\n\007schemas\030\003 \001(" - "\003\"J\n\013MetricValue\022\021\n\tmetric_id\030\001 \001(\005\022\022\n\nl" - "ong_value\030\002 \001(\003\022\024\n\014double_value\030\003 \001(\001\")\n" - "\010Registry\022\035\n\003jar\030\001 \003(\0132\020.exec.shared.Jar" - "\"/\n\003Jar\022\014\n\004name\030\001 \001(\t\022\032\n\022function_signat" - "ure\030\002 \003(\t\"W\n\013SaslMessage\022\021\n\tmechanism\030\001 " - "\001(\t\022\014\n\004data\030\002 \001(\014\022\'\n\006status\030\003 \001(\0162\027.exec" - ".shared.SaslStatus*5\n\nRpcChannel\022\017\n\013BIT_" - "CONTROL\020\000\022\014\n\010BIT_DATA\020\001\022\010\n\004USER\020\002*V\n\tQue" - "ryType\022\007\n\003SQL\020\001\022\013\n\007LOGICAL\020\002\022\014\n\010PHYSICAL" - "\020\003\022\r\n\tEXECUTION\020\004\022\026\n\022PREPARED_STATEMENT\020" - "\005*\207\001\n\rFragmentState\022\013\n\007SENDING\020\000\022\027\n\023AWAI" - "TING_ALLOCATION\020\001\022\013\n\007RUNNING\020\002\022\014\n\010FINISH" - "ED\020\003\022\r\n\tCANCELLED\020\004\022\n\n\006FAILED\020\005\022\032\n\026CANCE" - "LLATION_REQUESTED\020\006*\374\t\n\020CoreOperatorType" - "\022\021\n\rSINGLE_SENDER\020\000\022\024\n\020BROADCAST_SENDER\020" - "\001\022\n\n\006FILTER\020\002\022\022\n\016HASH_AGGREGATE\020\003\022\r\n\tHAS" - "H_JOIN\020\004\022\016\n\nMERGE_JOIN\020\005\022\031\n\025HASH_PARTITI" - "ON_SENDER\020\006\022\t\n\005LIMIT\020\007\022\024\n\020MERGING_RECEIV" - "ER\020\010\022\034\n\030ORDERED_PARTITION_SENDER\020\t\022\013\n\007PR" - "OJECT\020\n\022\026\n\022UNORDERED_RECEIVER\020\013\022\032\n\026RANGE" - "_PARTITION_SENDER\020\014\022\n\n\006SCREEN\020\r\022\034\n\030SELEC" - "TION_VECTOR_REMOVER\020\016\022\027\n\023STREAMING_AGGRE" - "GATE\020\017\022\016\n\nTOP_N_SORT\020\020\022\021\n\rEXTERNAL_SORT\020" - "\021\022\t\n\005TRACE\020\022\022\t\n\005UNION\020\023\022\014\n\010OLD_SORT\020\024\022\032\n" - "\026PARQUET_ROW_GROUP_SCAN\020\025\022\021\n\rHIVE_SUB_SC" - "AN\020\026\022\025\n\021SYSTEM_TABLE_SCAN\020\027\022\021\n\rMOCK_SUB_" - "SCAN\020\030\022\022\n\016PARQUET_WRITER\020\031\022\023\n\017DIRECT_SUB" - "_SCAN\020\032\022\017\n\013TEXT_WRITER\020\033\022\021\n\rTEXT_SUB_SCA" - "N\020\034\022\021\n\rJSON_SUB_SCAN\020\035\022\030\n\024INFO_SCHEMA_SU" - "B_SCAN\020\036\022\023\n\017COMPLEX_TO_JSON\020\037\022\025\n\021PRODUCE" - "R_CONSUMER\020 \022\022\n\016HBASE_SUB_SCAN\020!\022\n\n\006WIND" - "OW\020\"\022\024\n\020NESTED_LOOP_JOIN\020#\022\021\n\rAVRO_SUB_S" - "CAN\020$\022\021\n\rPCAP_SUB_SCAN\020%\022\022\n\016KAFKA_SUB_SC" - "AN\020&\022\021\n\rKUDU_SUB_SCAN\020\'\022\013\n\007FLATTEN\020(\022\020\n\014" - "LATERAL_JOIN\020)\022\n\n\006UNNEST\020*\022,\n(HIVE_DRILL" - "_NATIVE_PARQUET_ROW_GROUP_SCAN\020+\022\r\n\tJDBC" - "_SCAN\020,\022\022\n\016REGEX_SUB_SCAN\020-\022\023\n\017MAPRDB_SU" - "B_SCAN\020.\022\022\n\016MONGO_SUB_SCAN\020/\022\017\n\013KUDU_WRI" - "TER\0200\022\026\n\022OPEN_TSDB_SUB_SCAN\0201\022\017\n\013JSON_WR" - "ITER\0202\022\026\n\022HTPPD_LOG_SUB_SCAN\0203\022\022\n\016IMAGE_" - "SUB_SCAN\0204\022\025\n\021SEQUENCE_SUB_SCAN\0205\022\023\n\017PAR" - "TITION_LIMIT\0206\022\023\n\017PCAPNG_SUB_SCAN\0207\022\022\n\016R" - "UNTIME_FILTER\0208\022\017\n\013ROWKEY_JOIN\0209\022\023\n\017SYSL" - "OG_SUB_SCAN\020:\022\030\n\024STATISTICS_AGGREGATE\020;\022" - "\020\n\014UNPIVOT_MAPS\020<\022\024\n\020STATISTICS_MERGE\020=\022" - "\021\n\rLTSV_SUB_SCAN\020>*g\n\nSaslStatus\022\020\n\014SASL" - "_UNKNOWN\020\000\022\016\n\nSASL_START\020\001\022\024\n\020SASL_IN_PR" - "OGRESS\020\002\022\020\n\014SASL_SUCCESS\020\003\022\017\n\013SASL_FAILE" - "D\020\004B.\n\033org.apache.drill.exec.protoB\rUser" - "BitSharedH\001" + "e\022\022\n\nwait_nanos\030\t \001(\003\022\026\n\016max_allocation\030" + "\n \001(\003\"B\n\rStreamProfile\022\017\n\007records\030\001 \001(\003\022" + "\017\n\007batches\030\002 \001(\003\022\017\n\007schemas\030\003 \001(\003\"J\n\013Met" + "ricValue\022\021\n\tmetric_id\030\001 \001(\005\022\022\n\nlong_valu" + "e\030\002 \001(\003\022\024\n\014double_value\030\003 \001(\001\")\n\010Registr" + "y\022\035\n\003jar\030\001 \003(\0132\020.exec.shared.Jar\"/\n\003Jar\022" + "\014\n\004name\030\001 \001(\t\022\032\n\022function_signature\030\002 \003(" + "\t\"W\n\013SaslMessage\022\021\n\tmechanism\030\001 \001(\t\022\014\n\004d" + "ata\030\002 \001(\014\022\'\n\006status\030\003 \001(\0162\027.exec.shared." + "SaslStatus*5\n\nRpcChannel\022\017\n\013BIT_CONTROL\020" + "\000\022\014\n\010BIT_DATA\020\001\022\010\n\004USER\020\002*V\n\tQueryType\022\007" + "\n\003SQL\020\001\022\013\n\007LOGICAL\020\002\022\014\n\010PHYSICAL\020\003\022\r\n\tEX" + "ECUTION\020\004\022\026\n\022PREPARED_STATEMENT\020\005*\207\001\n\rFr" + "agmentState\022\013\n\007SENDING\020\000\022\027\n\023AWAITING_ALL" + "OCATION\020\001\022\013\n\007RUNNING\020\002\022\014\n\010FINISHED\020\003\022\r\n\t" + "CANCELLED\020\004\022\n\n\006FAILED\020\005\022\032\n\026CANCELLATION_" + "REQUESTED\020\006*\374\t\n\020CoreOperatorType\022\021\n\rSING" + "LE_SENDER\020\000\022\024\n\020BROADCAST_SENDER\020\001\022\n\n\006FIL" + "TER\020\002\022\022\n\016HASH_AGGREGATE\020\003\022\r\n\tHASH_JOIN\020\004" + "\022\016\n\nMERGE_JOIN\020\005\022\031\n\025HASH_PARTITION_SENDE" + "R\020\006\022\t\n\005LIMIT\020\007\022\024\n\020MERGING_RECEIVER\020\010\022\034\n\030" + "ORDERED_PARTITION_SENDER\020\t\022\013\n\007PROJECT\020\n\022" + "\026\n\022UNORDERED_RECEIVER\020\013\022\032\n\026RANGE_PARTITI" + "ON_SENDER\020\014\022\n\n\006SCREEN\020\r\022\034\n\030SELECTION_VEC" + "TOR_REMOVER\020\016\022\027\n\023STREAMING_AGGREGATE\020\017\022\016" + "\n\nTOP_N_SORT\020\020\022\021\n\rEXTERNAL_SORT\020\021\022\t\n\005TRA" + "CE\020\022\022\t\n\005UNION\020\023\022\014\n\010OLD_SORT\020\024\022\032\n\026PARQUET" + "_ROW_GROUP_SCAN\020\025\022\021\n\rHIVE_SUB_SCAN\020\026\022\025\n\021" + "SYSTEM_TABLE_SCAN\020\027\022\021\n\rMOCK_SUB_SCAN\020\030\022\022" + "\n\016PARQUET_WRITER\020\031\022\023\n\017DIRECT_SUB_SCAN\020\032\022" + "\017\n\013TEXT_WRITER\020\033\022\021\n\rTEXT_SUB_SCAN\020\034\022\021\n\rJ" + "SON_SUB_SCAN\020\035\022\030\n\024INFO_SCHEMA_SUB_SCAN\020\036" + "\022\023\n\017COMPLEX_TO_JSON\020\037\022\025\n\021PRODUCER_CONSUM" + "ER\020 \022\022\n\016HBASE_SUB_SCAN\020!\022\n\n\006WINDOW\020\"\022\024\n\020" + "NESTED_LOOP_JOIN\020#\022\021\n\rAVRO_SUB_SCAN\020$\022\021\n" + "\rPCAP_SUB_SCAN\020%\022\022\n\016KAFKA_SUB_SCAN\020&\022\021\n\r" + "KUDU_SUB_SCAN\020\'\022\013\n\007FLATTEN\020(\022\020\n\014LATERAL_" + "JOIN\020)\022\n\n\006UNNEST\020*\022,\n(HIVE_DRILL_NATIVE_" + "PARQUET_ROW_GROUP_SCAN\020+\022\r\n\tJDBC_SCAN\020,\022" + "\022\n\016REGEX_SUB_SCAN\020-\022\023\n\017MAPRDB_SUB_SCAN\020." + "\022\022\n\016MONGO_SUB_SCAN\020/\022\017\n\013KUDU_WRITER\0200\022\026\n" + "\022OPEN_TSDB_SUB_SCAN\0201\022\017\n\013JSON_WRITER\0202\022\026" + "\n\022HTPPD_LOG_SUB_SCAN\0203\022\022\n\016IMAGE_SUB_SCAN" + "\0204\022\025\n\021SEQUENCE_SUB_SCAN\0205\022\023\n\017PARTITION_L" + "IMIT\0206\022\023\n\017PCAPNG_SUB_SCAN\0207\022\022\n\016RUNTIME_F" + "ILTER\0208\022\017\n\013ROWKEY_JOIN\0209\022\023\n\017SYSLOG_SUB_S" + "CAN\020:\022\030\n\024STATISTICS_AGGREGATE\020;\022\020\n\014UNPIV" + "OT_MAPS\020<\022\024\n\020STATISTICS_MERGE\020=\022\021\n\rLTSV_" + "SUB_SCAN\020>*g\n\nSaslStatus\022\020\n\014SASL_UNKNOWN" + "\020\000\022\016\n\nSASL_START\020\001\022\024\n\020SASL_IN_PROGRESS\020\002" + "\022\020\n\014SASL_SUCCESS\020\003\022\017\n\013SASL_FAILED\020\004B.\n\033o" + "rg.apache.drill.exec.protoB\rUserBitShare" + "dH\001" }; ::google::protobuf::DescriptorPool::InternalAddGeneratedFile( - descriptor, 5691); + descriptor, 5683); ::google::protobuf::MessageFactory::InternalRegisterGeneratedFile( "UserBitShared.proto", &protobuf_RegisterTypes); ::protobuf_Types_2eproto::AddDescriptors(); @@ -8687,7 +8687,7 @@ const int OperatorProfile::kProcessNanosFieldNumber; const int OperatorProfile::kPeakLocalMemoryAllocatedFieldNumber; const int OperatorProfile::kMetricFieldNumber; const int OperatorProfile::kWaitNanosFieldNumber; -const int OperatorProfile::kOptimalMemAllocationFieldNumber; +const int OperatorProfile::kMaxAllocationFieldNumber; #endif // !defined(_MSC_VER) || _MSC_VER >= 1900 OperatorProfile::OperatorProfile() @@ -8705,15 +8705,15 @@ OperatorProfile::OperatorProfile(const OperatorProfile& from) metric_(from.metric_) { _internal_metadata_.MergeFrom(from._internal_metadata_); ::memcpy(&operator_id_, &from.operator_id_, - static_cast(reinterpret_cast(&optimal_mem_allocation_) - - reinterpret_cast(&operator_id_)) + sizeof(optimal_mem_allocation_)); + static_cast(reinterpret_cast(&max_allocation_) - + reinterpret_cast(&operator_id_)) + sizeof(max_allocation_)); // @@protoc_insertion_point(copy_constructor:exec.shared.OperatorProfile) } void OperatorProfile::SharedCtor() { ::memset(&operator_id_, 0, static_cast( - reinterpret_cast(&optimal_mem_allocation_) - - reinterpret_cast(&operator_id_)) + sizeof(optimal_mem_allocation_)); + reinterpret_cast(&max_allocation_) - + reinterpret_cast(&operator_id_)) + sizeof(max_allocation_)); } OperatorProfile::~OperatorProfile() { @@ -8749,8 +8749,8 @@ void OperatorProfile::Clear() { cached_has_bits = _has_bits_[0]; if (cached_has_bits & 127u) { ::memset(&operator_id_, 0, static_cast( - reinterpret_cast(&optimal_mem_allocation_) - - reinterpret_cast(&operator_id_)) + sizeof(optimal_mem_allocation_)); + reinterpret_cast(&max_allocation_) - + reinterpret_cast(&operator_id_)) + sizeof(max_allocation_)); } _has_bits_.Clear(); _internal_metadata_.Clear(); @@ -8874,14 +8874,14 @@ bool OperatorProfile::MergePartialFromCodedStream( break; } - // optional int64 optimal_mem_allocation = 10; + // optional int64 max_allocation = 10; case 10: { if (static_cast< ::google::protobuf::uint8>(tag) == static_cast< ::google::protobuf::uint8>(80u /* 80 & 0xFF */)) { - set_has_optimal_mem_allocation(); + set_has_max_allocation(); DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive< ::google::protobuf::int64, ::google::protobuf::internal::WireFormatLite::TYPE_INT64>( - input, &optimal_mem_allocation_))); + input, &max_allocation_))); } else { goto handle_unusual; } @@ -8963,9 +8963,9 @@ void OperatorProfile::SerializeWithCachedSizes( ::google::protobuf::internal::WireFormatLite::WriteInt64(9, this->wait_nanos(), output); } - // optional int64 optimal_mem_allocation = 10; + // optional int64 max_allocation = 10; if (cached_has_bits & 0x00000040u) { - ::google::protobuf::internal::WireFormatLite::WriteInt64(10, this->optimal_mem_allocation(), output); + ::google::protobuf::internal::WireFormatLite::WriteInt64(10, this->max_allocation(), output); } if (_internal_metadata_.have_unknown_fields()) { @@ -9029,9 +9029,9 @@ ::google::protobuf::uint8* OperatorProfile::InternalSerializeWithCachedSizesToAr target = ::google::protobuf::internal::WireFormatLite::WriteInt64ToArray(9, this->wait_nanos(), target); } - // optional int64 optimal_mem_allocation = 10; + // optional int64 max_allocation = 10; if (cached_has_bits & 0x00000040u) { - target = ::google::protobuf::internal::WireFormatLite::WriteInt64ToArray(10, this->optimal_mem_allocation(), target); + target = ::google::protobuf::internal::WireFormatLite::WriteInt64ToArray(10, this->max_allocation(), target); } if (_internal_metadata_.have_unknown_fields()) { @@ -9116,11 +9116,11 @@ size_t OperatorProfile::ByteSizeLong() const { this->wait_nanos()); } - // optional int64 optimal_mem_allocation = 10; - if (has_optimal_mem_allocation()) { + // optional int64 max_allocation = 10; + if (has_max_allocation()) { total_size += 1 + ::google::protobuf::internal::WireFormatLite::Int64Size( - this->optimal_mem_allocation()); + this->max_allocation()); } } @@ -9174,7 +9174,7 @@ void OperatorProfile::MergeFrom(const OperatorProfile& from) { wait_nanos_ = from.wait_nanos_; } if (cached_has_bits & 0x00000040u) { - optimal_mem_allocation_ = from.optimal_mem_allocation_; + max_allocation_ = from.max_allocation_; } _has_bits_[0] |= cached_has_bits; } @@ -9212,7 +9212,7 @@ void OperatorProfile::InternalSwap(OperatorProfile* other) { swap(process_nanos_, other->process_nanos_); swap(peak_local_memory_allocated_, other->peak_local_memory_allocated_); swap(wait_nanos_, other->wait_nanos_); - swap(optimal_mem_allocation_, other->optimal_mem_allocation_); + swap(max_allocation_, other->max_allocation_); swap(_has_bits_[0], other->_has_bits_[0]); _internal_metadata_.Swap(&other->_internal_metadata_); } diff --git a/contrib/native/client/src/protobuf/UserBitShared.pb.h b/contrib/native/client/src/protobuf/UserBitShared.pb.h index 1d1e2ffddb1..7e78abbe119 100644 --- a/contrib/native/client/src/protobuf/UserBitShared.pb.h +++ b/contrib/native/client/src/protobuf/UserBitShared.pb.h @@ -3504,12 +3504,12 @@ class OperatorProfile : public ::google::protobuf::Message /* @@protoc_insertion ::google::protobuf::int64 wait_nanos() const; void set_wait_nanos(::google::protobuf::int64 value); - // optional int64 optimal_mem_allocation = 10; - bool has_optimal_mem_allocation() const; - void clear_optimal_mem_allocation(); - static const int kOptimalMemAllocationFieldNumber = 10; - ::google::protobuf::int64 optimal_mem_allocation() const; - void set_optimal_mem_allocation(::google::protobuf::int64 value); + // optional int64 max_allocation = 10; + bool has_max_allocation() const; + void clear_max_allocation(); + static const int kMaxAllocationFieldNumber = 10; + ::google::protobuf::int64 max_allocation() const; + void set_max_allocation(::google::protobuf::int64 value); // @@protoc_insertion_point(class_scope:exec.shared.OperatorProfile) private: @@ -3525,8 +3525,8 @@ class OperatorProfile : public ::google::protobuf::Message /* @@protoc_insertion void clear_has_peak_local_memory_allocated(); void set_has_wait_nanos(); void clear_has_wait_nanos(); - void set_has_optimal_mem_allocation(); - void clear_has_optimal_mem_allocation(); + void set_has_max_allocation(); + void clear_has_max_allocation(); ::google::protobuf::internal::InternalMetadataWithArena _internal_metadata_; ::google::protobuf::internal::HasBits<1> _has_bits_; @@ -3539,7 +3539,7 @@ class OperatorProfile : public ::google::protobuf::Message /* @@protoc_insertion ::google::protobuf::int64 process_nanos_; ::google::protobuf::int64 peak_local_memory_allocated_; ::google::protobuf::int64 wait_nanos_; - ::google::protobuf::int64 optimal_mem_allocation_; + ::google::protobuf::int64 max_allocation_; friend struct ::protobuf_UserBitShared_2eproto::TableStruct; }; // ------------------------------------------------------------------- @@ -8090,28 +8090,28 @@ inline void OperatorProfile::set_wait_nanos(::google::protobuf::int64 value) { // @@protoc_insertion_point(field_set:exec.shared.OperatorProfile.wait_nanos) } -// optional int64 optimal_mem_allocation = 10; -inline bool OperatorProfile::has_optimal_mem_allocation() const { +// optional int64 max_allocation = 10; +inline bool OperatorProfile::has_max_allocation() const { return (_has_bits_[0] & 0x00000040u) != 0; } -inline void OperatorProfile::set_has_optimal_mem_allocation() { +inline void OperatorProfile::set_has_max_allocation() { _has_bits_[0] |= 0x00000040u; } -inline void OperatorProfile::clear_has_optimal_mem_allocation() { +inline void OperatorProfile::clear_has_max_allocation() { _has_bits_[0] &= ~0x00000040u; } -inline void OperatorProfile::clear_optimal_mem_allocation() { - optimal_mem_allocation_ = GOOGLE_LONGLONG(0); - clear_has_optimal_mem_allocation(); +inline void OperatorProfile::clear_max_allocation() { + max_allocation_ = GOOGLE_LONGLONG(0); + clear_has_max_allocation(); } -inline ::google::protobuf::int64 OperatorProfile::optimal_mem_allocation() const { - // @@protoc_insertion_point(field_get:exec.shared.OperatorProfile.optimal_mem_allocation) - return optimal_mem_allocation_; +inline ::google::protobuf::int64 OperatorProfile::max_allocation() const { + // @@protoc_insertion_point(field_get:exec.shared.OperatorProfile.max_allocation) + return max_allocation_; } -inline void OperatorProfile::set_optimal_mem_allocation(::google::protobuf::int64 value) { - set_has_optimal_mem_allocation(); - optimal_mem_allocation_ = value; - // @@protoc_insertion_point(field_set:exec.shared.OperatorProfile.optimal_mem_allocation) +inline void OperatorProfile::set_max_allocation(::google::protobuf::int64 value) { + set_has_max_allocation(); + max_allocation_ = value; + // @@protoc_insertion_point(field_set:exec.shared.OperatorProfile.max_allocation) } // ------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/java/org/apache/drill/common/DrillNode.java b/exec/java-exec/src/main/java/org/apache/drill/common/DrillNode.java index 8af70a3132c..1860126d550 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/common/DrillNode.java +++ b/exec/java-exec/src/main/java/org/apache/drill/common/DrillNode.java @@ -35,17 +35,38 @@ public static DrillNode create(DrillbitEndpoint endpoint) { return new DrillNode(endpoint); } - public boolean equals(Object other) { - if (!(other instanceof DrillNode)) { - return false; + public boolean equals(Object obj) { + if (obj == this) { + return true; } + if (!(obj instanceof DrillNode)) { + return super.equals(obj); + } + DrillbitEndpoint other = ((DrillNode)obj).endpoint; - DrillbitEndpoint otherEndpoint = ((DrillNode) other).endpoint; - return endpoint.getAddress().equals(otherEndpoint.getAddress()) && - endpoint.getUserPort() == otherEndpoint.getUserPort() && - endpoint.getControlPort() == otherEndpoint.getControlPort() && - endpoint.getDataPort() == otherEndpoint.getDataPort() && - endpoint.getVersion().equals(otherEndpoint.getVersion()); + boolean result = true; + result = result && (endpoint.hasAddress() == other.hasAddress()); + if (endpoint.hasAddress()) { + result = result && endpoint.getAddress() + .equals(other.getAddress()); + } + result = result && (endpoint.hasUserPort() == other.hasUserPort()); + if (endpoint.hasUserPort()) { + result = result && (endpoint.getUserPort() == other.getUserPort()); + } + result = result && (endpoint.hasControlPort() == other.hasControlPort()); + if (endpoint.hasControlPort()) { + result = result && (endpoint.getControlPort() == other.getControlPort()); + } + result = result && (endpoint.hasDataPort() == other.hasDataPort()); + if (endpoint.hasDataPort()) { + result = result && (endpoint.getDataPort() == other.getDataPort()); + } + result = result && (endpoint.hasVersion() == other.hasVersion()); + if (endpoint.hasVersion()) { + result = result && endpoint.getVersion().equals(other.getVersion()); + } + return result; } @Override @@ -81,8 +102,8 @@ public String toString() { StringBuilder sb = new StringBuilder(); return sb.append("endpoint address :") - .append(endpoint.getAddress()) + .append(endpoint.hasAddress() ? endpoint.getAddress() : "no-address") .append("endpoint user port: ") - .append(endpoint.getUserPort()).toString(); + .append(endpoint.hasUserPort() ? endpoint.getUserPort() : "no-userport").toString(); } } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorStats.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorStats.java index f682104aa1a..8061eab67fc 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorStats.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorStats.java @@ -88,7 +88,7 @@ public OperatorStats(OperatorStats original, boolean isClean) { } @VisibleForTesting - public OperatorStats(int operatorId, int operatorType, int inputCount, BufferAllocator allocator, long initialAllocation) { + public OperatorStats(int operatorId, int operatorType, int inputCount, BufferAllocator allocator, long optimalMemAllocation) { super(); this.allocator = allocator; this.operatorId = operatorId; @@ -97,7 +97,7 @@ public OperatorStats(int operatorId, int operatorType, int inputCount, BufferAll this.recordsReceivedByInput = new long[inputCount]; this.batchesReceivedByInput = new long[inputCount]; this.schemaCountByInput = new long[inputCount]; - this.optimalMemoryAllocation = initialAllocation; + this.optimalMemoryAllocation = optimalMemAllocation; } private String assertionError(String msg){ @@ -208,7 +208,7 @@ public OperatorProfile getProfile() { .setOperatorId(operatorId) // .setSetupNanos(setupNanos) // .setProcessNanos(processingNanos) - .setOptimalMemAllocation(optimalMemoryAllocation) + .setMaxAllocation(optimalMemoryAllocation) .setWaitNanos(waitNanos); if (allocator != null) { 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 541130b1670..3c12b187496 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 @@ -47,7 +47,7 @@ * fragment is based on the cluster state and provided queue configuration. */ public class DistributedQueueParallelizer extends SimpleParallelizer { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DistributedQueueParallelizer.class); + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DistributedQueueParallelizer.class); private final boolean planHasMemory; private final QueryContext queryContext; private final QueryResourceManager rm; @@ -62,23 +62,21 @@ public DistributedQueueParallelizer(boolean memoryPlanning, QueryContext queryCo } // return the memory computed for a physical operator on a drillbitendpoint. + // At this stage buffered operator memory could have been reduced depending upon + // the selected queue limits. public BiFunction getMemory() { return (endpoint, operator) -> { + long operatorsMemory = operator.getMaxAllocation(); if (!planHasMemory) { - final DrillNode drillEndpointNode = DrillNode.create(endpoint); if (operator.isBufferedOperator(queryContext)) { - Long operatorsMemory = operators.get(drillEndpointNode).get(operator); - logger.debug(" Memory requirement for the operator {} in endpoint {} is {}", operator, endpoint, operatorsMemory); - return operatorsMemory; + final DrillNode drillEndpointNode = DrillNode.create(endpoint); + operatorsMemory = operators.get(drillEndpointNode).get(operator); } else { - Long nonBufferedMemory = (long)operator.getCost().getMemoryCost(); - logger.debug(" Memory requirement for the operator {} in endpoint {} is {}", operator, endpoint, nonBufferedMemory); - return nonBufferedMemory; + operatorsMemory = (long)operator.getCost().getMemoryCost(); } } - else { - return operator.getMaxAllocation(); - } + logger.debug(" Memory requirement for the operator {} in endpoint {} is {}", operator, endpoint, operatorsMemory); + return operatorsMemory; }; } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Fragment.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Fragment.java index 3d28067d563..cb4f8d15993 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Fragment.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Fragment.java @@ -18,9 +18,9 @@ package org.apache.drill.exec.planner.fragment; import org.apache.drill.exec.ops.QueryContext; -import org.apache.drill.exec.physical.base.AbstractPhysicalVisitor; import org.apache.drill.exec.physical.base.Exchange; import org.apache.drill.exec.physical.base.PhysicalOperator; +import org.apache.drill.exec.util.memory.MemoryAllocationUtilities; import org.apache.drill.exec.work.foreman.ForemanSetupException; import org.apache.drill.shaded.guava.com.google.common.collect.Lists; @@ -167,27 +167,11 @@ public boolean equals(Object obj) { public List getBufferedOperators(QueryContext queryContext) { List bufferedOps = new ArrayList<>(); - root.accept(new BufferedOpFinder(queryContext), bufferedOps); + root.accept(new MemoryAllocationUtilities.BufferedOpFinder(queryContext), bufferedOps); return bufferedOps; } - protected static class BufferedOpFinder extends AbstractPhysicalVisitor, RuntimeException> { - private final QueryContext context; - public BufferedOpFinder(QueryContext queryContext) { - this.context = queryContext; - } - - @Override - public Void visitOp(PhysicalOperator op, List value) - throws RuntimeException { - if (op.isBufferedOperator(context)) { - value.add(op); - } - visitChildren(op, value); - return null; - } - } @Override public String toString() { 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 4593c55f6ad..508036a129a 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 @@ -49,13 +49,13 @@ public class MemoryCalculator extends AbstractOpWrapperVisitor>> bufferedOperators; private final QueryContext queryContext; - private final long MINIMUM_MEMORY_FOR_BUFFER_OPERS; + private final long minimum_memory_for_buffer_opers; public MemoryCalculator(PlanningSet planningSet, QueryContext context, long minMemory) { this.planningSet = planningSet; this.bufferedOperators = new HashMap<>(); this.queryContext = context; - this.MINIMUM_MEMORY_FOR_BUFFER_OPERS = minMemory; + this.minimum_memory_for_buffer_opers = minMemory; } // Helper method to compute the minor fragment count per drillbit. This method returns @@ -138,7 +138,7 @@ public Void visitOp(PhysicalOperator op, Wrapper fragment) { // The memory estimates of the optimizer are for the whole operator spread across all the // minor fragments. Divide this memory estimation by fragment width to get the memory // requirement per minor fragment. - long memoryCostPerMinorFrag = Math.max((long)Math.ceil(memoryCost/fragment.getAssignedEndpoints().size()), MINIMUM_MEMORY_FOR_BUFFER_OPERS); + long memoryCostPerMinorFrag = Math.max((long)Math.ceil(memoryCost/fragment.getAssignedEndpoints().size()), minimum_memory_for_buffer_opers); Map drillbitEndpointMinorFragMap = getMinorFragCountPerDrillbit(fragment); Map roots, public class Collector extends AbstractOpWrapperVisitor { - private final Multimap bufferedOperators; + private final Map> bufferedOperators; public Collector() { - this.bufferedOperators = ArrayListMultimap.create(); + this.bufferedOperators = new HashMap<>(); } private void getMinorFragCountPerDrillbit(Wrapper currFragment, PhysicalOperator operator) { for (DrillbitEndpoint endpoint : currFragment.getAssignedEndpoints()) { - bufferedOperators.put(endpoint, operator); + DrillNode node = new DrillNode(endpoint); + bufferedOperators.putIfAbsent(node, new ArrayList<>()); + bufferedOperators.get(node).add(operator); } } @@ -103,10 +107,10 @@ public Void visitOp(PhysicalOperator op, Wrapper fragment) { } public Map> getNodeMap() { - Map> endpointCollectionMap = bufferedOperators.asMap(); + Map> endpointCollectionMap = bufferedOperators; Map> nodeMap = new HashMap<>(); - for (Map.Entry> entry : endpointCollectionMap.entrySet()) { - nodeMap.put(entry.getKey().getAddress(), entry.getValue()); + for (Map.Entry> entry : endpointCollectionMap.entrySet()) { + nodeMap.put(entry.getKey().toString(), entry.getValue()); } return nodeMap; 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 5b92688f8ea..c3139162936 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 @@ -44,18 +44,18 @@ public class NodeResources { private long memoryInBytes; - private long numVirtualCpu; + private int numVirtualCpu; private static final int CURRENT_VERSION = 1; - public NodeResources(long memoryInBytes, long numVirtualCpu) { + public NodeResources(long memoryInBytes, int numVirtualCpu) { this(CURRENT_VERSION, memoryInBytes, numVirtualCpu); } @JsonCreator public NodeResources(@JsonProperty("version") int version, @JsonProperty("memoryInBytes") long memoryInBytes, - @JsonProperty("numVirtualCpu") long numVirtualCpu) { + @JsonProperty("numVirtualCpu") int numVirtualCpu) { this.version = version; this.memoryInBytes = memoryInBytes; this.numVirtualCpu = numVirtualCpu; @@ -79,7 +79,7 @@ public long getMemoryInGB() { return Math.round(getMemoryInMB() / 1024L); } - public long getNumVirtualCpu() { + public int getNumVirtualCpu() { return numVirtualCpu; } @@ -140,11 +140,11 @@ public static NodeResources create() { return create(0,0); } - public static NodeResources create(long cpu) { + public static NodeResources create(int cpu) { return create(cpu,0); } - public static NodeResources create(long cpu, long memory) { + public static NodeResources create(int cpu, long memory) { return new NodeResources(CURRENT_VERSION, memory, cpu); } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/util/memory/DefaultMemoryAllocationUtilities.java b/exec/java-exec/src/main/java/org/apache/drill/exec/util/memory/DefaultMemoryAllocationUtilities.java index 648276ef201..a7861cd2b35 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/util/memory/DefaultMemoryAllocationUtilities.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/util/memory/DefaultMemoryAllocationUtilities.java @@ -17,40 +17,18 @@ */ package org.apache.drill.exec.util.memory; -import java.util.ArrayList; import java.util.List; import org.apache.drill.common.config.DrillConfig; -import org.apache.drill.exec.ExecConstants; -import org.apache.drill.exec.memory.RootAllocatorFactory; import org.apache.drill.exec.ops.QueryContext; -import org.apache.drill.exec.physical.PhysicalPlan; import org.apache.drill.exec.physical.base.PhysicalOperator; import org.apache.drill.exec.server.options.OptionManager; -import org.apache.drill.exec.server.options.OptionSet; -import org.apache.drill.shaded.guava.com.google.common.annotations.VisibleForTesting; - -public class DefaultMemoryAllocationUtilities { +public class DefaultMemoryAllocationUtilities extends MemoryAllocationUtilities { private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DefaultMemoryAllocationUtilities.class); - public static void setupBufferedMemoryAllocations(PhysicalPlan plan, final QueryContext queryContext) { - setupBufferedOpsMemoryAllocations(plan.getProperties().hasResourcePlan, - getBufferedOperators(plan.getSortedOperators(), queryContext), queryContext); - } - - public static List getBufferedOperators(List operators, QueryContext queryContext) { - final List bufferedOpList = new ArrayList<>(); - for (final PhysicalOperator op : operators) { - if (op.isBufferedOperator(queryContext)) { - bufferedOpList.add(op); - } - } - return bufferedOpList; - } - /** * Helper method to setup Memory Allocations *

    @@ -105,65 +83,5 @@ public static void setupBufferedOpsMemoryAllocations(boolean planHasMemory, } } - /** - * Compute per-operator memory based on the computed per-node memory, the - * number of operators, and the computed number of fragments (which house - * the operators.) Enforces a floor on the amount of memory per operator. - * - * @param optionManager system option manager - * @param maxAllocPerNode computed query memory per node - * @param opCount number of buffering operators in this query - * @return the per-operator memory - */ - - public static long computeOperatorMemory(OptionSet optionManager, long maxAllocPerNode, int opCount) { - final long maxWidth = optionManager.getOption(ExecConstants.MAX_WIDTH_PER_NODE); - final double cpuLoadAverage = optionManager.getOption(ExecConstants.CPU_LOAD_AVERAGE); - final long maxWidthPerNode = ExecConstants.MAX_WIDTH_PER_NODE.computeMaxWidth(cpuLoadAverage, maxWidth); - final long maxOperatorAlloc = maxAllocPerNode / (opCount * maxWidthPerNode); - logger.debug("Max buffered operator alloc: {}", maxOperatorAlloc); - - // User configurable option to allow forcing minimum memory. - // Ensure that the buffered ops receive the minimum memory needed to make progress. - // Without this, the math might work out to allocate too little memory. - return Math.max(maxOperatorAlloc, - optionManager.getOption(ExecConstants.MIN_MEMORY_PER_BUFFERED_OP)); - } - - /** - * Per-node memory calculations based on a number of constraints. - *

    - * Factored out into a separate method to allow unit testing. - * @param config Drill config - * @param optionManager system options - * @param directMemory amount of direct memory - * @return memory per query per node - */ - - @VisibleForTesting - public static long computeQueryMemory(DrillConfig config, OptionSet optionManager, long directMemory) { - - // Memory computed as a percent of total memory. - - long perQueryMemory = Math.round(directMemory * - optionManager.getOption(ExecConstants.PERCENT_MEMORY_PER_QUERY)); - - // But, must allow at least the amount given explicitly for - // backward compatibility. - - perQueryMemory = Math.max(perQueryMemory, - optionManager.getOption(ExecConstants.MAX_QUERY_MEMORY_PER_NODE)); - - // Compute again as either the total direct memory, or the - // configured maximum top-level allocation (10 GB). - - long maxAllocPerNode = Math.min(directMemory, - config.getLong(RootAllocatorFactory.TOP_LEVEL_MAX_ALLOC)); - - // Final amount per node per query is the minimum of these two. - - maxAllocPerNode = Math.min(maxAllocPerNode, perQueryMemory); - return maxAllocPerNode; - } } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/util/memory/MemoryAllocationUtilities.java b/exec/java-exec/src/main/java/org/apache/drill/exec/util/memory/MemoryAllocationUtilities.java new file mode 100644 index 00000000000..c74e3be56a5 --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/util/memory/MemoryAllocationUtilities.java @@ -0,0 +1,113 @@ +/* + * 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.util.memory; + +import org.apache.drill.common.config.DrillConfig; +import org.apache.drill.exec.ExecConstants; +import org.apache.drill.exec.memory.RootAllocatorFactory; +import org.apache.drill.exec.ops.QueryContext; +import org.apache.drill.exec.physical.base.AbstractPhysicalVisitor; +import org.apache.drill.exec.physical.base.PhysicalOperator; +import org.apache.drill.exec.server.options.OptionSet; +import org.apache.drill.shaded.guava.com.google.common.annotations.VisibleForTesting; + +import java.util.List; + +public class MemoryAllocationUtilities { + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MemoryAllocationUtilities.class); + /** + * Per-node memory calculations based on a number of constraints. + *

    + * Factored out into a separate method to allow unit testing. + * @param config Drill config + * @param optionManager system options + * @param directMemory amount of direct memory + * @return memory per query per node + */ + + @VisibleForTesting + public static long computeQueryMemory(DrillConfig config, OptionSet optionManager, long directMemory) { + + // Memory computed as a percent of total memory. + + long perQueryMemory = Math.round(directMemory * + optionManager.getOption(ExecConstants.PERCENT_MEMORY_PER_QUERY)); + + // But, must allow at least the amount given explicitly for + // backward compatibility. + + perQueryMemory = Math.max(perQueryMemory, + optionManager.getOption(ExecConstants.MAX_QUERY_MEMORY_PER_NODE)); + + // Compute again as either the total direct memory, or the + // configured maximum top-level allocation (10 GB). + + long maxAllocPerNode = Math.min(directMemory, + config.getLong(RootAllocatorFactory.TOP_LEVEL_MAX_ALLOC)); + + // Final amount per node per query is the minimum of these two. + + maxAllocPerNode = Math.min(maxAllocPerNode, perQueryMemory); + return maxAllocPerNode; + } + + public static class BufferedOpFinder extends AbstractPhysicalVisitor, RuntimeException> { + private final QueryContext context; + + public BufferedOpFinder(QueryContext queryContext) { + this.context = queryContext; + } + + @Override + public Void visitOp(PhysicalOperator op, List value) + throws RuntimeException { + if (op.isBufferedOperator(context)) { + value.add(op); + } + visitChildren(op, value); + return null; + } + } + + + /** + * Compute per-operator memory based on the computed per-node memory, the + * number of operators, and the computed number of fragments (which house + * the operators.) Enforces a floor on the amount of memory per operator. + * + * @param optionManager system option manager + * @param maxAllocPerNode computed query memory per node + * @param opCount number of buffering operators in this query + * @return the per-operator memory + */ + + public static long computeOperatorMemory(OptionSet optionManager, long maxAllocPerNode, int opCount) { + final long maxWidth = optionManager.getOption(ExecConstants.MAX_WIDTH_PER_NODE); + final double cpuLoadAverage = optionManager.getOption(ExecConstants.CPU_LOAD_AVERAGE); + final long maxWidthPerNode = ExecConstants.MAX_WIDTH_PER_NODE.computeMaxWidth(cpuLoadAverage, maxWidth); + final long maxOperatorAlloc = maxAllocPerNode / (opCount * maxWidthPerNode); + logger.debug("Max buffered operator alloc: {}", maxOperatorAlloc); + + // User configurable option to allow forcing minimum memory. + // Ensure that the buffered ops receive the minimum memory needed to make progress. + // Without this, the math might work out to allocate too little memory. + + return Math.max(maxOperatorAlloc, + optionManager.getOption(ExecConstants.MIN_MEMORY_PER_BUFFERED_OP)); + } +} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/util/memory/ZKQueueMemoryAllocationUtilities.java b/exec/java-exec/src/main/java/org/apache/drill/exec/util/memory/ZKQueueMemoryAllocationUtilities.java index 6d232596bdf..f38d19e9074 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/util/memory/ZKQueueMemoryAllocationUtilities.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/util/memory/ZKQueueMemoryAllocationUtilities.java @@ -17,27 +17,14 @@ */ package org.apache.drill.exec.util.memory; -import org.apache.drill.common.config.DrillConfig; -import org.apache.drill.exec.ExecConstants; -import org.apache.drill.exec.memory.RootAllocatorFactory; import org.apache.drill.exec.ops.QueryContext; -import org.apache.drill.exec.physical.base.AbstractPhysicalVisitor; -import org.apache.drill.exec.physical.base.FragmentRoot; import org.apache.drill.exec.physical.base.PhysicalOperator; import org.apache.drill.exec.proto.helper.QueryIdHelper; -import org.apache.drill.exec.server.options.OptionSet; -import org.apache.drill.exec.work.QueryWorkUnit; import org.apache.drill.exec.work.foreman.rm.QueryResourceManager; -import org.apache.drill.shaded.guava.com.google.common.annotations.VisibleForTesting; -import org.apache.drill.shaded.guava.com.google.common.collect.ArrayListMultimap; -import org.apache.drill.shaded.guava.com.google.common.collect.Multimap; - -import java.util.ArrayList; import java.util.Collection; -import java.util.List; import java.util.Map; -public class ZKQueueMemoryAllocationUtilities { +public class ZKQueueMemoryAllocationUtilities extends MemoryAllocationUtilities { private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ZKQueueMemoryAllocationUtilities.class); public static void planMemory(QueryContext queryContext, QueryResourceManager rm, Map> nodeMap) { @@ -59,59 +46,6 @@ public static void planMemory(QueryContext queryContext, QueryResourceManager rm } } - public static void getBufferedOps(Multimap map, - QueryWorkUnit.MinorFragmentDefn defn) { - List bufferedOps = getBufferedOps(defn.root()); - if (!bufferedOps.isEmpty()) { - map.putAll(defn.fragment().getAssignment().getAddress(), bufferedOps); - } - } - - public static class BufferedOpFinder extends AbstractPhysicalVisitor, RuntimeException> { - @Override - public Void visitOp(PhysicalOperator op, List value) - throws RuntimeException { - if (op.isBufferedOperator(null)) { - value.add(op); - } - visitChildren(op, value); - return null; - } - } - - /** - * Search an individual fragment tree to find any buffered operators it may - * contain. - * - * @param root - * @return - */ - - private static List getBufferedOps(FragmentRoot root) { - List bufferedOps = new ArrayList<>(); - BufferedOpFinder finder = new BufferedOpFinder(); - root.accept(finder, bufferedOps); - return bufferedOps; - } - - /** - * Build a list of external sorts grouped by node. We start with a list of - * minor fragments, each with an endpoint (node). Multiple minor fragments - * may appear on each node, and each minor fragment may have 0, 1 or more - * sorts. - * - * @return - */ - - private static Map> buildBufferedOpMap(QueryWorkUnit work) { - Multimap map = ArrayListMultimap.create(); - getBufferedOps(map, work.getRootFragmentDefn()); - for (QueryWorkUnit.MinorFragmentDefn defn : work.getMinorFragmentDefns()) { - getBufferedOps(map, defn); - } - return map.asMap(); - } - private static int countBufferingOperators(Map> nodeMap) { int width = 0; for (Collection fragSorts : nodeMap.values()) { @@ -191,66 +125,4 @@ else if (alloc < preferredOpMemory) { op.setMaxAllocation(alloc); } } - - /** - * Compute per-operator memory based on the computed per-node memory, the - * number of operators, and the computed number of fragments (which house - * the operators.) Enforces a floor on the amount of memory per operator. - * - * @param optionManager system option manager - * @param maxAllocPerNode computed query memory per node - * @param opCount number of buffering operators in this query - * @return the per-operator memory - */ - - public static long computeOperatorMemory(OptionSet optionManager, long maxAllocPerNode, int opCount) { - final long maxWidth = optionManager.getOption(ExecConstants.MAX_WIDTH_PER_NODE); - final double cpuLoadAverage = optionManager.getOption(ExecConstants.CPU_LOAD_AVERAGE); - final long maxWidthPerNode = ExecConstants.MAX_WIDTH_PER_NODE.computeMaxWidth(cpuLoadAverage, maxWidth); - final long maxOperatorAlloc = maxAllocPerNode / (opCount * maxWidthPerNode); - logger.debug("Max buffered operator alloc: {}", maxOperatorAlloc); - - // User configurable option to allow forcing minimum memory. - // Ensure that the buffered ops receive the minimum memory needed to make progress. - // Without this, the math might work out to allocate too little memory. - - return Math.max(maxOperatorAlloc, - optionManager.getOption(ExecConstants.MIN_MEMORY_PER_BUFFERED_OP)); - } - - /** - * Per-node memory calculations based on a number of constraints. - *

    - * Factored out into a separate method to allow unit testing. - * @param config Drill config - * @param optionManager system options - * @param directMemory amount of direct memory - * @return memory per query per node - */ - - @VisibleForTesting - public static long computeQueryMemory(DrillConfig config, OptionSet optionManager, long directMemory) { - - // Memory computed as a percent of total memory. - - long perQueryMemory = Math.round(directMemory * - optionManager.getOption(ExecConstants.PERCENT_MEMORY_PER_QUERY)); - - // But, must allow at least the amount given explicitly for - // backward compatibility. - - perQueryMemory = Math.max(perQueryMemory, - optionManager.getOption(ExecConstants.MAX_QUERY_MEMORY_PER_NODE)); - - // Compute again as either the total direct memory, or the - // configured maximum top-level allocation (10 GB). - - long maxAllocPerNode = Math.min(directMemory, - config.getLong(RootAllocatorFactory.TOP_LEVEL_MAX_ALLOC)); - - // Final amount per node per query is the minimum of these two. - - maxAllocPerNode = Math.min(maxAllocPerNode, perQueryMemory); - return maxAllocPerNode; - } } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/user/PlanSplitter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/user/PlanSplitter.java index 9a853899a11..956ccf97325 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/user/PlanSplitter.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/user/PlanSplitter.java @@ -132,7 +132,6 @@ private List getFragments(final DrillbitContext dContext, final Ge final QueryWorkUnit queryWorkUnit = parallelizer.generateWorkUnit(queryContext.getOptions().getOptionList(), queryContext.getCurrentEndpoint(), queryId, queryContext.getOnlineEndpointUUIDs(), rootFragment, queryContext.getSession(), queryContext.getQueryContextInfo()); -// planner.visitPhysicalPlan(queryWorkUnit); queryWorkUnit.applyPlan(dContext.getPlanReader()); fragments.add(queryWorkUnit.getRootFragment()); fragments.addAll(queryWorkUnit.getFragments()); diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/BitControl.java b/protocol/src/main/java/org/apache/drill/exec/proto/BitControl.java index 63ce995d1e5..c961a748f68 100644 --- a/protocol/src/main/java/org/apache/drill/exec/proto/BitControl.java +++ b/protocol/src/main/java/org/apache/drill/exec/proto/BitControl.java @@ -4493,18 +4493,18 @@ org.apache.drill.exec.proto.BitControl.CollectorOrBuilder getCollectorOrBuilder( int index); /** - * optional string endpointUUID = 18; + * optional string assignedEndpointUUID = 18; */ - boolean hasEndpointUUID(); + boolean hasAssignedEndpointUUID(); /** - * optional string endpointUUID = 18; + * optional string assignedEndpointUUID = 18; */ - java.lang.String getEndpointUUID(); + java.lang.String getAssignedEndpointUUID(); /** - * optional string endpointUUID = 18; + * optional string assignedEndpointUUID = 18; */ com.google.protobuf.ByteString - getEndpointUUIDBytes(); + getAssignedEndpointUUIDBytes(); } /** * Protobuf type {@code exec.bit.control.PlanFragment} @@ -4529,7 +4529,7 @@ private PlanFragment() { memMax_ = 2000000000L; optionsJson_ = ""; collector_ = java.util.Collections.emptyList(); - endpointUUID_ = ""; + assignedEndpointUUID_ = ""; } @java.lang.Override @@ -4680,7 +4680,7 @@ private PlanFragment( case 146: { com.google.protobuf.ByteString bs = input.readBytes(); bitField0_ |= 0x00004000; - endpointUUID_ = bs; + assignedEndpointUUID_ = bs; break; } default: { @@ -5064,19 +5064,19 @@ public org.apache.drill.exec.proto.BitControl.CollectorOrBuilder getCollectorOrB return collector_.get(index); } - public static final int ENDPOINTUUID_FIELD_NUMBER = 18; - private volatile java.lang.Object endpointUUID_; + public static final int ASSIGNEDENDPOINTUUID_FIELD_NUMBER = 18; + private volatile java.lang.Object assignedEndpointUUID_; /** - * optional string endpointUUID = 18; + * optional string assignedEndpointUUID = 18; */ - public boolean hasEndpointUUID() { + public boolean hasAssignedEndpointUUID() { return ((bitField0_ & 0x00004000) == 0x00004000); } /** - * optional string endpointUUID = 18; + * optional string assignedEndpointUUID = 18; */ - public java.lang.String getEndpointUUID() { - java.lang.Object ref = endpointUUID_; + public java.lang.String getAssignedEndpointUUID() { + java.lang.Object ref = assignedEndpointUUID_; if (ref instanceof java.lang.String) { return (java.lang.String) ref; } else { @@ -5084,22 +5084,22 @@ public java.lang.String getEndpointUUID() { (com.google.protobuf.ByteString) ref; java.lang.String s = bs.toStringUtf8(); if (bs.isValidUtf8()) { - endpointUUID_ = s; + assignedEndpointUUID_ = s; } return s; } } /** - * optional string endpointUUID = 18; + * optional string assignedEndpointUUID = 18; */ public com.google.protobuf.ByteString - getEndpointUUIDBytes() { - java.lang.Object ref = endpointUUID_; + getAssignedEndpointUUIDBytes() { + java.lang.Object ref = assignedEndpointUUID_; if (ref instanceof java.lang.String) { com.google.protobuf.ByteString b = com.google.protobuf.ByteString.copyFromUtf8( (java.lang.String) ref); - endpointUUID_ = b; + assignedEndpointUUID_ = b; return b; } else { return (com.google.protobuf.ByteString) ref; @@ -5166,7 +5166,7 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) output.writeMessage(17, collector_.get(i)); } if (((bitField0_ & 0x00004000) == 0x00004000)) { - com.google.protobuf.GeneratedMessageV3.writeString(output, 18, endpointUUID_); + com.google.protobuf.GeneratedMessageV3.writeString(output, 18, assignedEndpointUUID_); } unknownFields.writeTo(output); } @@ -5236,7 +5236,7 @@ public int getSerializedSize() { .computeMessageSize(17, collector_.get(i)); } if (((bitField0_ & 0x00004000) == 0x00004000)) { - size += com.google.protobuf.GeneratedMessageV3.computeStringSize(18, endpointUUID_); + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(18, assignedEndpointUUID_); } size += unknownFields.getSerializedSize(); memoizedSize = size; @@ -5334,10 +5334,10 @@ public boolean equals(final java.lang.Object obj) { } result = result && getCollectorList() .equals(other.getCollectorList()); - result = result && (hasEndpointUUID() == other.hasEndpointUUID()); - if (hasEndpointUUID()) { - result = result && getEndpointUUID() - .equals(other.getEndpointUUID()); + result = result && (hasAssignedEndpointUUID() == other.hasAssignedEndpointUUID()); + if (hasAssignedEndpointUUID()) { + result = result && getAssignedEndpointUUID() + .equals(other.getAssignedEndpointUUID()); } result = result && unknownFields.equals(other.unknownFields); return result; @@ -5417,9 +5417,9 @@ public int hashCode() { hash = (37 * hash) + COLLECTOR_FIELD_NUMBER; hash = (53 * hash) + getCollectorList().hashCode(); } - if (hasEndpointUUID()) { - hash = (37 * hash) + ENDPOINTUUID_FIELD_NUMBER; - hash = (53 * hash) + getEndpointUUID().hashCode(); + if (hasAssignedEndpointUUID()) { + hash = (37 * hash) + ASSIGNEDENDPOINTUUID_FIELD_NUMBER; + hash = (53 * hash) + getAssignedEndpointUUID().hashCode(); } hash = (29 * hash) + unknownFields.hashCode(); memoizedHashCode = hash; @@ -5614,7 +5614,7 @@ public Builder clear() { } else { collectorBuilder_.clear(); } - endpointUUID_ = ""; + assignedEndpointUUID_ = ""; bitField0_ = (bitField0_ & ~0x00008000); return this; } @@ -5732,7 +5732,7 @@ public org.apache.drill.exec.proto.BitControl.PlanFragment buildPartial() { if (((from_bitField0_ & 0x00008000) == 0x00008000)) { to_bitField0_ |= 0x00004000; } - result.endpointUUID_ = endpointUUID_; + result.assignedEndpointUUID_ = assignedEndpointUUID_; result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -5854,9 +5854,9 @@ public Builder mergeFrom(org.apache.drill.exec.proto.BitControl.PlanFragment oth } } } - if (other.hasEndpointUUID()) { + if (other.hasAssignedEndpointUUID()) { bitField0_ |= 0x00008000; - endpointUUID_ = other.endpointUUID_; + assignedEndpointUUID_ = other.assignedEndpointUUID_; onChanged(); } this.mergeUnknownFields(other.unknownFields); @@ -7127,24 +7127,24 @@ public org.apache.drill.exec.proto.BitControl.Collector.Builder addCollectorBuil return collectorBuilder_; } - private java.lang.Object endpointUUID_ = ""; + private java.lang.Object assignedEndpointUUID_ = ""; /** - * optional string endpointUUID = 18; + * optional string assignedEndpointUUID = 18; */ - public boolean hasEndpointUUID() { + public boolean hasAssignedEndpointUUID() { return ((bitField0_ & 0x00008000) == 0x00008000); } /** - * optional string endpointUUID = 18; + * optional string assignedEndpointUUID = 18; */ - public java.lang.String getEndpointUUID() { - java.lang.Object ref = endpointUUID_; + public java.lang.String getAssignedEndpointUUID() { + java.lang.Object ref = assignedEndpointUUID_; if (!(ref instanceof java.lang.String)) { com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; java.lang.String s = bs.toStringUtf8(); if (bs.isValidUtf8()) { - endpointUUID_ = s; + assignedEndpointUUID_ = s; } return s; } else { @@ -7152,53 +7152,53 @@ public java.lang.String getEndpointUUID() { } } /** - * optional string endpointUUID = 18; + * optional string assignedEndpointUUID = 18; */ public com.google.protobuf.ByteString - getEndpointUUIDBytes() { - java.lang.Object ref = endpointUUID_; + getAssignedEndpointUUIDBytes() { + java.lang.Object ref = assignedEndpointUUID_; if (ref instanceof String) { com.google.protobuf.ByteString b = com.google.protobuf.ByteString.copyFromUtf8( (java.lang.String) ref); - endpointUUID_ = b; + assignedEndpointUUID_ = b; return b; } else { return (com.google.protobuf.ByteString) ref; } } /** - * optional string endpointUUID = 18; + * optional string assignedEndpointUUID = 18; */ - public Builder setEndpointUUID( + public Builder setAssignedEndpointUUID( java.lang.String value) { if (value == null) { throw new NullPointerException(); } bitField0_ |= 0x00008000; - endpointUUID_ = value; + assignedEndpointUUID_ = value; onChanged(); return this; } /** - * optional string endpointUUID = 18; + * optional string assignedEndpointUUID = 18; */ - public Builder clearEndpointUUID() { + public Builder clearAssignedEndpointUUID() { bitField0_ = (bitField0_ & ~0x00008000); - endpointUUID_ = getDefaultInstance().getEndpointUUID(); + assignedEndpointUUID_ = getDefaultInstance().getAssignedEndpointUUID(); onChanged(); return this; } /** - * optional string endpointUUID = 18; + * optional string assignedEndpointUUID = 18; */ - public Builder setEndpointUUIDBytes( + public Builder setAssignedEndpointUUIDBytes( com.google.protobuf.ByteString value) { if (value == null) { throw new NullPointerException(); } bitField0_ |= 0x00008000; - endpointUUID_ = value; + assignedEndpointUUID_ = value; onChanged(); return this; } @@ -10872,7 +10872,7 @@ public org.apache.drill.exec.proto.BitControl.FinishedReceiver getDefaultInstanc "c.bit.FragmentHandle\"G\n\023InitializeFragme" + "nts\0220\n\010fragment\030\001 \003(\0132\036.exec.bit.control" + ".PlanFragment\".\n\rCustomMessage\022\014\n\004type\030\001" + - " \001(\005\022\017\n\007message\030\002 \001(\014\"\222\004\n\014PlanFragment\022(" + + " \001(\005\022\017\n\007message\030\002 \001(\014\"\232\004\n\014PlanFragment\022(" + "\n\006handle\030\001 \001(\0132\030.exec.bit.FragmentHandle" + "\022\024\n\014network_cost\030\004 \001(\002\022\020\n\010cpu_cost\030\005 \001(\002" + "\022\021\n\tdisk_cost\030\006 \001(\002\022\023\n\013memory_cost\030\007 \001(\002" + @@ -10885,30 +10885,30 @@ public org.apache.drill.exec.proto.BitControl.FinishedReceiver getDefaultInstanc "ls\022\024\n\014options_json\030\017 \001(\t\022:\n\007context\030\020 \001(" + "\0132).exec.bit.control.QueryContextInforma" + "tion\022.\n\tcollector\030\021 \003(\0132\033.exec.bit.contr" + - "ol.Collector\022\024\n\014endpointUUID\030\022 \001(\t\"\210\001\n\tC" + - "ollector\022\"\n\032opposite_major_fragment_id\030\001" + - " \001(\005\022#\n\027incoming_minor_fragment\030\002 \003(\005B\002\020" + - "\001\022\035\n\025supports_out_of_order\030\003 \001(\010\022\023\n\013is_s" + - "pooling\030\004 \001(\010\"w\n\027QueryContextInformation" + - "\022\030\n\020query_start_time\030\001 \001(\003\022\021\n\ttime_zone\030" + - "\002 \001(\005\022\033\n\023default_schema_name\030\003 \001(\t\022\022\n\nse" + - "ssion_id\030\004 \001(\t\"f\n\017WorkQueueStatus\022(\n\010end" + - "point\030\001 \001(\0132\026.exec.DrillbitEndpoint\022\024\n\014q" + - "ueue_length\030\002 \001(\005\022\023\n\013report_time\030\003 \001(\003\"h" + - "\n\020FinishedReceiver\022*\n\010receiver\030\001 \001(\0132\030.e" + - "xec.bit.FragmentHandle\022(\n\006sender\030\002 \001(\0132\030" + - ".exec.bit.FragmentHandle*\206\003\n\007RpcType\022\r\n\t" + - "HANDSHAKE\020\000\022\007\n\003ACK\020\001\022\013\n\007GOODBYE\020\002\022\034\n\030REQ" + - "_INITIALIZE_FRAGMENTS\020\003\022\027\n\023REQ_CANCEL_FR" + - "AGMENT\020\006\022\031\n\025REQ_RECEIVER_FINISHED\020\007\022\027\n\023R" + - "EQ_FRAGMENT_STATUS\020\010\022\022\n\016REQ_BIT_STATUS\020\t" + - "\022\024\n\020REQ_QUERY_STATUS\020\n\022\024\n\020REQ_QUERY_CANC" + - "EL\020\017\022\030\n\024REQ_UNPAUSE_FRAGMENT\020\020\022\016\n\nREQ_CU" + - "STOM\020\021\022\030\n\024RESP_FRAGMENT_HANDLE\020\013\022\030\n\024RESP" + - "_FRAGMENT_STATUS\020\014\022\023\n\017RESP_BIT_STATUS\020\r\022" + - "\025\n\021RESP_QUERY_STATUS\020\016\022\017\n\013RESP_CUSTOM\020\022\022" + - "\020\n\014SASL_MESSAGE\020\023B+\n\033org.apache.drill.ex" + - "ec.protoB\nBitControlH\001" + "ol.Collector\022\034\n\024assignedEndpointUUID\030\022 \001" + + "(\t\"\210\001\n\tCollector\022\"\n\032opposite_major_fragm" + + "ent_id\030\001 \001(\005\022#\n\027incoming_minor_fragment\030" + + "\002 \003(\005B\002\020\001\022\035\n\025supports_out_of_order\030\003 \001(\010" + + "\022\023\n\013is_spooling\030\004 \001(\010\"w\n\027QueryContextInf" + + "ormation\022\030\n\020query_start_time\030\001 \001(\003\022\021\n\tti" + + "me_zone\030\002 \001(\005\022\033\n\023default_schema_name\030\003 \001" + + "(\t\022\022\n\nsession_id\030\004 \001(\t\"f\n\017WorkQueueStatu" + + "s\022(\n\010endpoint\030\001 \001(\0132\026.exec.DrillbitEndpo" + + "int\022\024\n\014queue_length\030\002 \001(\005\022\023\n\013report_time" + + "\030\003 \001(\003\"h\n\020FinishedReceiver\022*\n\010receiver\030\001" + + " \001(\0132\030.exec.bit.FragmentHandle\022(\n\006sender" + + "\030\002 \001(\0132\030.exec.bit.FragmentHandle*\206\003\n\007Rpc" + + "Type\022\r\n\tHANDSHAKE\020\000\022\007\n\003ACK\020\001\022\013\n\007GOODBYE\020" + + "\002\022\034\n\030REQ_INITIALIZE_FRAGMENTS\020\003\022\027\n\023REQ_C" + + "ANCEL_FRAGMENT\020\006\022\031\n\025REQ_RECEIVER_FINISHE" + + "D\020\007\022\027\n\023REQ_FRAGMENT_STATUS\020\010\022\022\n\016REQ_BIT_" + + "STATUS\020\t\022\024\n\020REQ_QUERY_STATUS\020\n\022\024\n\020REQ_QU" + + "ERY_CANCEL\020\017\022\030\n\024REQ_UNPAUSE_FRAGMENT\020\020\022\016" + + "\n\nREQ_CUSTOM\020\021\022\030\n\024RESP_FRAGMENT_HANDLE\020\013" + + "\022\030\n\024RESP_FRAGMENT_STATUS\020\014\022\023\n\017RESP_BIT_S" + + "TATUS\020\r\022\025\n\021RESP_QUERY_STATUS\020\016\022\017\n\013RESP_C" + + "USTOM\020\022\022\020\n\014SASL_MESSAGE\020\023B+\n\033org.apache." + + "drill.exec.protoB\nBitControlH\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor. InternalDescriptorAssigner() { @@ -10960,7 +10960,7 @@ public com.google.protobuf.ExtensionRegistry assignDescriptors( internal_static_exec_bit_control_PlanFragment_fieldAccessorTable = new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( internal_static_exec_bit_control_PlanFragment_descriptor, - new java.lang.String[] { "Handle", "NetworkCost", "CpuCost", "DiskCost", "MemoryCost", "FragmentJson", "LeafFragment", "Assignment", "Foreman", "MemInitial", "MemMax", "Credentials", "OptionsJson", "Context", "Collector", "EndpointUUID", }); + new java.lang.String[] { "Handle", "NetworkCost", "CpuCost", "DiskCost", "MemoryCost", "FragmentJson", "LeafFragment", "Assignment", "Foreman", "MemInitial", "MemMax", "Credentials", "OptionsJson", "Context", "Collector", "AssignedEndpointUUID", }); internal_static_exec_bit_control_Collector_descriptor = getDescriptor().getMessageTypes().get(6); internal_static_exec_bit_control_Collector_fieldAccessorTable = new diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/SchemaBitControl.java b/protocol/src/main/java/org/apache/drill/exec/proto/SchemaBitControl.java index 735549f1d5b..e4c67be98df 100644 --- a/protocol/src/main/java/org/apache/drill/exec/proto/SchemaBitControl.java +++ b/protocol/src/main/java/org/apache/drill/exec/proto/SchemaBitControl.java @@ -672,8 +672,8 @@ public void writeTo(com.dyuproject.protostuff.Output output, org.apache.drill.ex for(org.apache.drill.exec.proto.BitControl.Collector collector : message.getCollectorList()) output.writeObject(17, collector, org.apache.drill.exec.proto.SchemaBitControl.Collector.WRITE, true); - if(message.hasEndpointUUID()) - output.writeString(18, message.getEndpointUUID(), false); + if(message.hasAssignedEndpointUUID()) + output.writeString(18, message.getAssignedEndpointUUID(), false); } public boolean isInitialized(org.apache.drill.exec.proto.BitControl.PlanFragment message) { @@ -765,7 +765,7 @@ public void mergeFrom(com.dyuproject.protostuff.Input input, org.apache.drill.ex break; case 18: - builder.setEndpointUUID(input.readString()); + builder.setAssignedEndpointUUID(input.readString()); break; default: input.handleUnknownField(number, this); @@ -822,7 +822,7 @@ public static java.lang.String getFieldName(int number) case 15: return "optionsJson"; case 16: return "context"; case 17: return "collector"; - case 18: return "endpointUUID"; + case 18: return "assignedEndpointUUID"; default: return null; } } @@ -849,7 +849,7 @@ public static int getFieldNumber(java.lang.String name) fieldMap.put("optionsJson", 15); fieldMap.put("context", 16); fieldMap.put("collector", 17); - fieldMap.put("endpointUUID", 18); + fieldMap.put("assignedEndpointUUID", 18); } } diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/SchemaUserBitShared.java b/protocol/src/main/java/org/apache/drill/exec/proto/SchemaUserBitShared.java index 4babfb18148..e9c30ac2390 100644 --- a/protocol/src/main/java/org/apache/drill/exec/proto/SchemaUserBitShared.java +++ b/protocol/src/main/java/org/apache/drill/exec/proto/SchemaUserBitShared.java @@ -2377,8 +2377,8 @@ public void writeTo(com.dyuproject.protostuff.Output output, org.apache.drill.ex if(message.hasWaitNanos()) output.writeInt64(9, message.getWaitNanos(), false); - if(message.hasOptimalMemAllocation()) - output.writeInt64(10, message.getOptimalMemAllocation(), false); + if(message.hasMaxAllocation()) + output.writeInt64(10, message.getMaxAllocation(), false); } public boolean isInitialized(org.apache.drill.exec.proto.UserBitShared.OperatorProfile message) { @@ -2445,7 +2445,7 @@ public void mergeFrom(com.dyuproject.protostuff.Input input, org.apache.drill.ex builder.setWaitNanos(input.readInt64()); break; case 10: - builder.setOptimalMemAllocation(input.readInt64()); + builder.setMaxAllocation(input.readInt64()); break; default: input.handleUnknownField(number, this); @@ -2495,7 +2495,7 @@ public static java.lang.String getFieldName(int number) case 7: return "peakLocalMemoryAllocated"; case 8: return "metric"; case 9: return "waitNanos"; - case 10: return "optimalMemAllocation"; + case 10: return "maxAllocation"; default: return null; } } @@ -2515,7 +2515,7 @@ public static int getFieldNumber(java.lang.String name) fieldMap.put("peakLocalMemoryAllocated", 7); fieldMap.put("metric", 8); fieldMap.put("waitNanos", 9); - fieldMap.put("optimalMemAllocation", 10); + fieldMap.put("maxAllocation", 10); } } diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java b/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java index 452d90e3bda..e5fe2539f14 100644 --- a/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java +++ b/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java @@ -22365,13 +22365,13 @@ org.apache.drill.exec.proto.UserBitShared.MetricValueOrBuilder getMetricOrBuilde long getWaitNanos(); /** - * optional int64 optimal_mem_allocation = 10; + * optional int64 max_allocation = 10; */ - boolean hasOptimalMemAllocation(); + boolean hasMaxAllocation(); /** - * optional int64 optimal_mem_allocation = 10; + * optional int64 max_allocation = 10; */ - long getOptimalMemAllocation(); + long getMaxAllocation(); } /** * Protobuf type {@code exec.shared.OperatorProfile} @@ -22394,7 +22394,7 @@ private OperatorProfile() { peakLocalMemoryAllocated_ = 0L; metric_ = java.util.Collections.emptyList(); waitNanos_ = 0L; - optimalMemAllocation_ = 0L; + maxAllocation_ = 0L; } @java.lang.Override @@ -22471,7 +22471,7 @@ private OperatorProfile( } case 80: { bitField0_ |= 0x00000040; - optimalMemAllocation_ = input.readInt64(); + maxAllocation_ = input.readInt64(); break; } default: { @@ -22673,19 +22673,19 @@ public long getWaitNanos() { return waitNanos_; } - public static final int OPTIMAL_MEM_ALLOCATION_FIELD_NUMBER = 10; - private long optimalMemAllocation_; + public static final int MAX_ALLOCATION_FIELD_NUMBER = 10; + private long maxAllocation_; /** - * optional int64 optimal_mem_allocation = 10; + * optional int64 max_allocation = 10; */ - public boolean hasOptimalMemAllocation() { + public boolean hasMaxAllocation() { return ((bitField0_ & 0x00000040) == 0x00000040); } /** - * optional int64 optimal_mem_allocation = 10; + * optional int64 max_allocation = 10; */ - public long getOptimalMemAllocation() { - return optimalMemAllocation_; + public long getMaxAllocation() { + return maxAllocation_; } private byte memoizedIsInitialized = -1; @@ -22727,7 +22727,7 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) output.writeInt64(9, waitNanos_); } if (((bitField0_ & 0x00000040) == 0x00000040)) { - output.writeInt64(10, optimalMemAllocation_); + output.writeInt64(10, maxAllocation_); } unknownFields.writeTo(output); } @@ -22772,7 +22772,7 @@ public int getSerializedSize() { } if (((bitField0_ & 0x00000040) == 0x00000040)) { size += com.google.protobuf.CodedOutputStream - .computeInt64Size(10, optimalMemAllocation_); + .computeInt64Size(10, maxAllocation_); } size += unknownFields.getSerializedSize(); memoizedSize = size; @@ -22824,10 +22824,10 @@ public boolean equals(final java.lang.Object obj) { result = result && (getWaitNanos() == other.getWaitNanos()); } - result = result && (hasOptimalMemAllocation() == other.hasOptimalMemAllocation()); - if (hasOptimalMemAllocation()) { - result = result && (getOptimalMemAllocation() - == other.getOptimalMemAllocation()); + result = result && (hasMaxAllocation() == other.hasMaxAllocation()); + if (hasMaxAllocation()) { + result = result && (getMaxAllocation() + == other.getMaxAllocation()); } result = result && unknownFields.equals(other.unknownFields); return result; @@ -22876,10 +22876,10 @@ public int hashCode() { hash = (53 * hash) + com.google.protobuf.Internal.hashLong( getWaitNanos()); } - if (hasOptimalMemAllocation()) { - hash = (37 * hash) + OPTIMAL_MEM_ALLOCATION_FIELD_NUMBER; + if (hasMaxAllocation()) { + hash = (37 * hash) + MAX_ALLOCATION_FIELD_NUMBER; hash = (53 * hash) + com.google.protobuf.Internal.hashLong( - getOptimalMemAllocation()); + getMaxAllocation()); } hash = (29 * hash) + unknownFields.hashCode(); memoizedHashCode = hash; @@ -23040,7 +23040,7 @@ public Builder clear() { } waitNanos_ = 0L; bitField0_ = (bitField0_ & ~0x00000080); - optimalMemAllocation_ = 0L; + maxAllocation_ = 0L; bitField0_ = (bitField0_ & ~0x00000100); return this; } @@ -23115,7 +23115,7 @@ public org.apache.drill.exec.proto.UserBitShared.OperatorProfile buildPartial() if (((from_bitField0_ & 0x00000100) == 0x00000100)) { to_bitField0_ |= 0x00000040; } - result.optimalMemAllocation_ = optimalMemAllocation_; + result.maxAllocation_ = maxAllocation_; result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -23235,8 +23235,8 @@ public Builder mergeFrom(org.apache.drill.exec.proto.UserBitShared.OperatorProfi if (other.hasWaitNanos()) { setWaitNanos(other.getWaitNanos()); } - if (other.hasOptimalMemAllocation()) { - setOptimalMemAllocation(other.getOptimalMemAllocation()); + if (other.hasMaxAllocation()) { + setMaxAllocation(other.getMaxAllocation()); } this.mergeUnknownFields(other.unknownFields); onChanged(); @@ -23940,34 +23940,34 @@ public Builder clearWaitNanos() { return this; } - private long optimalMemAllocation_ ; + private long maxAllocation_ ; /** - * optional int64 optimal_mem_allocation = 10; + * optional int64 max_allocation = 10; */ - public boolean hasOptimalMemAllocation() { + public boolean hasMaxAllocation() { return ((bitField0_ & 0x00000100) == 0x00000100); } /** - * optional int64 optimal_mem_allocation = 10; + * optional int64 max_allocation = 10; */ - public long getOptimalMemAllocation() { - return optimalMemAllocation_; + public long getMaxAllocation() { + return maxAllocation_; } /** - * optional int64 optimal_mem_allocation = 10; + * optional int64 max_allocation = 10; */ - public Builder setOptimalMemAllocation(long value) { + public Builder setMaxAllocation(long value) { bitField0_ |= 0x00000100; - optimalMemAllocation_ = value; + maxAllocation_ = value; onChanged(); return this; } /** - * optional int64 optimal_mem_allocation = 10; + * optional int64 max_allocation = 10; */ - public Builder clearOptimalMemAllocation() { + public Builder clearMaxAllocation() { bitField0_ = (bitField0_ & ~0x00000100); - optimalMemAllocation_ = 0L; + maxAllocation_ = 0L; onChanged(); return this; } @@ -27946,65 +27946,65 @@ public org.apache.drill.exec.proto.UserBitShared.SaslMessage getDefaultInstanceF "y_used\030\007 \001(\003\022\027\n\017max_memory_used\030\010 \001(\003\022(\n" + "\010endpoint\030\t \001(\0132\026.exec.DrillbitEndpoint\022" + "\023\n\013last_update\030\n \001(\003\022\025\n\rlast_progress\030\013 " + - "\001(\003\"\237\002\n\017OperatorProfile\0221\n\rinput_profile" + + "\001(\003\"\227\002\n\017OperatorProfile\0221\n\rinput_profile" + "\030\001 \003(\0132\032.exec.shared.StreamProfile\022\023\n\013op" + "erator_id\030\003 \001(\005\022\025\n\roperator_type\030\004 \001(\005\022\023" + "\n\013setup_nanos\030\005 \001(\003\022\025\n\rprocess_nanos\030\006 \001" + "(\003\022#\n\033peak_local_memory_allocated\030\007 \001(\003\022" + "(\n\006metric\030\010 \003(\0132\030.exec.shared.MetricValu" + - "e\022\022\n\nwait_nanos\030\t \001(\003\022\036\n\026optimal_mem_all" + - "ocation\030\n \001(\003\"B\n\rStreamProfile\022\017\n\007record" + - "s\030\001 \001(\003\022\017\n\007batches\030\002 \001(\003\022\017\n\007schemas\030\003 \001(" + - "\003\"J\n\013MetricValue\022\021\n\tmetric_id\030\001 \001(\005\022\022\n\nl" + - "ong_value\030\002 \001(\003\022\024\n\014double_value\030\003 \001(\001\")\n" + - "\010Registry\022\035\n\003jar\030\001 \003(\0132\020.exec.shared.Jar" + - "\"/\n\003Jar\022\014\n\004name\030\001 \001(\t\022\032\n\022function_signat" + - "ure\030\002 \003(\t\"W\n\013SaslMessage\022\021\n\tmechanism\030\001 " + - "\001(\t\022\014\n\004data\030\002 \001(\014\022\'\n\006status\030\003 \001(\0162\027.exec" + - ".shared.SaslStatus*5\n\nRpcChannel\022\017\n\013BIT_" + - "CONTROL\020\000\022\014\n\010BIT_DATA\020\001\022\010\n\004USER\020\002*V\n\tQue" + - "ryType\022\007\n\003SQL\020\001\022\013\n\007LOGICAL\020\002\022\014\n\010PHYSICAL" + - "\020\003\022\r\n\tEXECUTION\020\004\022\026\n\022PREPARED_STATEMENT\020" + - "\005*\207\001\n\rFragmentState\022\013\n\007SENDING\020\000\022\027\n\023AWAI" + - "TING_ALLOCATION\020\001\022\013\n\007RUNNING\020\002\022\014\n\010FINISH" + - "ED\020\003\022\r\n\tCANCELLED\020\004\022\n\n\006FAILED\020\005\022\032\n\026CANCE" + - "LLATION_REQUESTED\020\006*\374\t\n\020CoreOperatorType" + - "\022\021\n\rSINGLE_SENDER\020\000\022\024\n\020BROADCAST_SENDER\020" + - "\001\022\n\n\006FILTER\020\002\022\022\n\016HASH_AGGREGATE\020\003\022\r\n\tHAS" + - "H_JOIN\020\004\022\016\n\nMERGE_JOIN\020\005\022\031\n\025HASH_PARTITI" + - "ON_SENDER\020\006\022\t\n\005LIMIT\020\007\022\024\n\020MERGING_RECEIV" + - "ER\020\010\022\034\n\030ORDERED_PARTITION_SENDER\020\t\022\013\n\007PR" + - "OJECT\020\n\022\026\n\022UNORDERED_RECEIVER\020\013\022\032\n\026RANGE" + - "_PARTITION_SENDER\020\014\022\n\n\006SCREEN\020\r\022\034\n\030SELEC" + - "TION_VECTOR_REMOVER\020\016\022\027\n\023STREAMING_AGGRE" + - "GATE\020\017\022\016\n\nTOP_N_SORT\020\020\022\021\n\rEXTERNAL_SORT\020" + - "\021\022\t\n\005TRACE\020\022\022\t\n\005UNION\020\023\022\014\n\010OLD_SORT\020\024\022\032\n" + - "\026PARQUET_ROW_GROUP_SCAN\020\025\022\021\n\rHIVE_SUB_SC" + - "AN\020\026\022\025\n\021SYSTEM_TABLE_SCAN\020\027\022\021\n\rMOCK_SUB_" + - "SCAN\020\030\022\022\n\016PARQUET_WRITER\020\031\022\023\n\017DIRECT_SUB" + - "_SCAN\020\032\022\017\n\013TEXT_WRITER\020\033\022\021\n\rTEXT_SUB_SCA" + - "N\020\034\022\021\n\rJSON_SUB_SCAN\020\035\022\030\n\024INFO_SCHEMA_SU" + - "B_SCAN\020\036\022\023\n\017COMPLEX_TO_JSON\020\037\022\025\n\021PRODUCE" + - "R_CONSUMER\020 \022\022\n\016HBASE_SUB_SCAN\020!\022\n\n\006WIND" + - "OW\020\"\022\024\n\020NESTED_LOOP_JOIN\020#\022\021\n\rAVRO_SUB_S" + - "CAN\020$\022\021\n\rPCAP_SUB_SCAN\020%\022\022\n\016KAFKA_SUB_SC" + - "AN\020&\022\021\n\rKUDU_SUB_SCAN\020\'\022\013\n\007FLATTEN\020(\022\020\n\014" + - "LATERAL_JOIN\020)\022\n\n\006UNNEST\020*\022,\n(HIVE_DRILL" + - "_NATIVE_PARQUET_ROW_GROUP_SCAN\020+\022\r\n\tJDBC" + - "_SCAN\020,\022\022\n\016REGEX_SUB_SCAN\020-\022\023\n\017MAPRDB_SU" + - "B_SCAN\020.\022\022\n\016MONGO_SUB_SCAN\020/\022\017\n\013KUDU_WRI" + - "TER\0200\022\026\n\022OPEN_TSDB_SUB_SCAN\0201\022\017\n\013JSON_WR" + - "ITER\0202\022\026\n\022HTPPD_LOG_SUB_SCAN\0203\022\022\n\016IMAGE_" + - "SUB_SCAN\0204\022\025\n\021SEQUENCE_SUB_SCAN\0205\022\023\n\017PAR" + - "TITION_LIMIT\0206\022\023\n\017PCAPNG_SUB_SCAN\0207\022\022\n\016R" + - "UNTIME_FILTER\0208\022\017\n\013ROWKEY_JOIN\0209\022\023\n\017SYSL" + - "OG_SUB_SCAN\020:\022\030\n\024STATISTICS_AGGREGATE\020;\022" + - "\020\n\014UNPIVOT_MAPS\020<\022\024\n\020STATISTICS_MERGE\020=\022" + - "\021\n\rLTSV_SUB_SCAN\020>*g\n\nSaslStatus\022\020\n\014SASL" + - "_UNKNOWN\020\000\022\016\n\nSASL_START\020\001\022\024\n\020SASL_IN_PR" + - "OGRESS\020\002\022\020\n\014SASL_SUCCESS\020\003\022\017\n\013SASL_FAILE" + - "D\020\004B.\n\033org.apache.drill.exec.protoB\rUser" + - "BitSharedH\001" + "e\022\022\n\nwait_nanos\030\t \001(\003\022\026\n\016max_allocation\030" + + "\n \001(\003\"B\n\rStreamProfile\022\017\n\007records\030\001 \001(\003\022" + + "\017\n\007batches\030\002 \001(\003\022\017\n\007schemas\030\003 \001(\003\"J\n\013Met" + + "ricValue\022\021\n\tmetric_id\030\001 \001(\005\022\022\n\nlong_valu" + + "e\030\002 \001(\003\022\024\n\014double_value\030\003 \001(\001\")\n\010Registr" + + "y\022\035\n\003jar\030\001 \003(\0132\020.exec.shared.Jar\"/\n\003Jar\022" + + "\014\n\004name\030\001 \001(\t\022\032\n\022function_signature\030\002 \003(" + + "\t\"W\n\013SaslMessage\022\021\n\tmechanism\030\001 \001(\t\022\014\n\004d" + + "ata\030\002 \001(\014\022\'\n\006status\030\003 \001(\0162\027.exec.shared." + + "SaslStatus*5\n\nRpcChannel\022\017\n\013BIT_CONTROL\020" + + "\000\022\014\n\010BIT_DATA\020\001\022\010\n\004USER\020\002*V\n\tQueryType\022\007" + + "\n\003SQL\020\001\022\013\n\007LOGICAL\020\002\022\014\n\010PHYSICAL\020\003\022\r\n\tEX" + + "ECUTION\020\004\022\026\n\022PREPARED_STATEMENT\020\005*\207\001\n\rFr" + + "agmentState\022\013\n\007SENDING\020\000\022\027\n\023AWAITING_ALL" + + "OCATION\020\001\022\013\n\007RUNNING\020\002\022\014\n\010FINISHED\020\003\022\r\n\t" + + "CANCELLED\020\004\022\n\n\006FAILED\020\005\022\032\n\026CANCELLATION_" + + "REQUESTED\020\006*\374\t\n\020CoreOperatorType\022\021\n\rSING" + + "LE_SENDER\020\000\022\024\n\020BROADCAST_SENDER\020\001\022\n\n\006FIL" + + "TER\020\002\022\022\n\016HASH_AGGREGATE\020\003\022\r\n\tHASH_JOIN\020\004" + + "\022\016\n\nMERGE_JOIN\020\005\022\031\n\025HASH_PARTITION_SENDE" + + "R\020\006\022\t\n\005LIMIT\020\007\022\024\n\020MERGING_RECEIVER\020\010\022\034\n\030" + + "ORDERED_PARTITION_SENDER\020\t\022\013\n\007PROJECT\020\n\022" + + "\026\n\022UNORDERED_RECEIVER\020\013\022\032\n\026RANGE_PARTITI" + + "ON_SENDER\020\014\022\n\n\006SCREEN\020\r\022\034\n\030SELECTION_VEC" + + "TOR_REMOVER\020\016\022\027\n\023STREAMING_AGGREGATE\020\017\022\016" + + "\n\nTOP_N_SORT\020\020\022\021\n\rEXTERNAL_SORT\020\021\022\t\n\005TRA" + + "CE\020\022\022\t\n\005UNION\020\023\022\014\n\010OLD_SORT\020\024\022\032\n\026PARQUET" + + "_ROW_GROUP_SCAN\020\025\022\021\n\rHIVE_SUB_SCAN\020\026\022\025\n\021" + + "SYSTEM_TABLE_SCAN\020\027\022\021\n\rMOCK_SUB_SCAN\020\030\022\022" + + "\n\016PARQUET_WRITER\020\031\022\023\n\017DIRECT_SUB_SCAN\020\032\022" + + "\017\n\013TEXT_WRITER\020\033\022\021\n\rTEXT_SUB_SCAN\020\034\022\021\n\rJ" + + "SON_SUB_SCAN\020\035\022\030\n\024INFO_SCHEMA_SUB_SCAN\020\036" + + "\022\023\n\017COMPLEX_TO_JSON\020\037\022\025\n\021PRODUCER_CONSUM" + + "ER\020 \022\022\n\016HBASE_SUB_SCAN\020!\022\n\n\006WINDOW\020\"\022\024\n\020" + + "NESTED_LOOP_JOIN\020#\022\021\n\rAVRO_SUB_SCAN\020$\022\021\n" + + "\rPCAP_SUB_SCAN\020%\022\022\n\016KAFKA_SUB_SCAN\020&\022\021\n\r" + + "KUDU_SUB_SCAN\020\'\022\013\n\007FLATTEN\020(\022\020\n\014LATERAL_" + + "JOIN\020)\022\n\n\006UNNEST\020*\022,\n(HIVE_DRILL_NATIVE_" + + "PARQUET_ROW_GROUP_SCAN\020+\022\r\n\tJDBC_SCAN\020,\022" + + "\022\n\016REGEX_SUB_SCAN\020-\022\023\n\017MAPRDB_SUB_SCAN\020." + + "\022\022\n\016MONGO_SUB_SCAN\020/\022\017\n\013KUDU_WRITER\0200\022\026\n" + + "\022OPEN_TSDB_SUB_SCAN\0201\022\017\n\013JSON_WRITER\0202\022\026" + + "\n\022HTPPD_LOG_SUB_SCAN\0203\022\022\n\016IMAGE_SUB_SCAN" + + "\0204\022\025\n\021SEQUENCE_SUB_SCAN\0205\022\023\n\017PARTITION_L" + + "IMIT\0206\022\023\n\017PCAPNG_SUB_SCAN\0207\022\022\n\016RUNTIME_F" + + "ILTER\0208\022\017\n\013ROWKEY_JOIN\0209\022\023\n\017SYSLOG_SUB_S" + + "CAN\020:\022\030\n\024STATISTICS_AGGREGATE\020;\022\020\n\014UNPIV" + + "OT_MAPS\020<\022\024\n\020STATISTICS_MERGE\020=\022\021\n\rLTSV_" + + "SUB_SCAN\020>*g\n\nSaslStatus\022\020\n\014SASL_UNKNOWN" + + "\020\000\022\016\n\nSASL_START\020\001\022\024\n\020SASL_IN_PROGRESS\020\002" + + "\022\020\n\014SASL_SUCCESS\020\003\022\017\n\013SASL_FAILED\020\004B.\n\033o" + + "rg.apache.drill.exec.protoB\rUserBitShare" + + "dH\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor. InternalDescriptorAssigner() { @@ -28122,7 +28122,7 @@ public com.google.protobuf.ExtensionRegistry assignDescriptors( internal_static_exec_shared_OperatorProfile_fieldAccessorTable = new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( internal_static_exec_shared_OperatorProfile_descriptor, - new java.lang.String[] { "InputProfile", "OperatorId", "OperatorType", "SetupNanos", "ProcessNanos", "PeakLocalMemoryAllocated", "Metric", "WaitNanos", "OptimalMemAllocation", }); + new java.lang.String[] { "InputProfile", "OperatorId", "OperatorType", "SetupNanos", "ProcessNanos", "PeakLocalMemoryAllocated", "Metric", "WaitNanos", "MaxAllocation", }); internal_static_exec_shared_StreamProfile_descriptor = getDescriptor().getMessageTypes().get(17); internal_static_exec_shared_StreamProfile_fieldAccessorTable = new diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/beans/OperatorProfile.java b/protocol/src/main/java/org/apache/drill/exec/proto/beans/OperatorProfile.java index 224214a69dd..a065a6cb8e0 100644 --- a/protocol/src/main/java/org/apache/drill/exec/proto/beans/OperatorProfile.java +++ b/protocol/src/main/java/org/apache/drill/exec/proto/beans/OperatorProfile.java @@ -57,7 +57,7 @@ public static OperatorProfile getDefaultInstance() private long peakLocalMemoryAllocated; private List metric; private long waitNanos; - private long optimalMemAllocation; + private long maxAllocation; public OperatorProfile() { @@ -170,16 +170,16 @@ public OperatorProfile setWaitNanos(long waitNanos) return this; } - // optimalMemAllocation + // maxAllocation - public long getOptimalMemAllocation() + public long getMaxAllocation() { - return optimalMemAllocation; + return maxAllocation; } - public OperatorProfile setOptimalMemAllocation(long optimalMemAllocation) + public OperatorProfile setMaxAllocation(long maxAllocation) { - this.optimalMemAllocation = optimalMemAllocation; + this.maxAllocation = maxAllocation; return this; } @@ -268,7 +268,7 @@ public void mergeFrom(Input input, OperatorProfile message) throws IOException message.waitNanos = input.readInt64(); break; case 10: - message.optimalMemAllocation = input.readInt64(); + message.maxAllocation = input.readInt64(); break; default: input.handleUnknownField(number, this); @@ -317,8 +317,8 @@ public void writeTo(Output output, OperatorProfile message) throws IOException if(message.waitNanos != 0) output.writeInt64(9, message.waitNanos, false); - if(message.optimalMemAllocation != 0) - output.writeInt64(10, message.optimalMemAllocation, false); + if(message.maxAllocation != 0) + output.writeInt64(10, message.maxAllocation, false); } public String getFieldName(int number) @@ -333,7 +333,7 @@ public String getFieldName(int number) case 7: return "peakLocalMemoryAllocated"; case 8: return "metric"; case 9: return "waitNanos"; - case 10: return "optimalMemAllocation"; + case 10: return "maxAllocation"; default: return null; } } @@ -355,7 +355,7 @@ public int getFieldNumber(String name) __fieldMap.put("peakLocalMemoryAllocated", 7); __fieldMap.put("metric", 8); __fieldMap.put("waitNanos", 9); - __fieldMap.put("optimalMemAllocation", 10); + __fieldMap.put("maxAllocation", 10); } } diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/beans/PlanFragment.java b/protocol/src/main/java/org/apache/drill/exec/proto/beans/PlanFragment.java index 7214f15882a..9afc892a2ad 100644 --- a/protocol/src/main/java/org/apache/drill/exec/proto/beans/PlanFragment.java +++ b/protocol/src/main/java/org/apache/drill/exec/proto/beans/PlanFragment.java @@ -66,7 +66,7 @@ public static PlanFragment getDefaultInstance() private String optionsJson; private QueryContextInformation context; private List collector; - private String endpointUUID; + private String assignedEndpointUUID; public PlanFragment() { @@ -270,16 +270,16 @@ public PlanFragment setCollectorList(List collector) return this; } - // endpointUUID + // assignedEndpointUUID - public String getEndpointUUID() + public String getAssignedEndpointUUID() { - return endpointUUID; + return assignedEndpointUUID; } - public PlanFragment setEndpointUUID(String endpointUUID) + public PlanFragment setAssignedEndpointUUID(String assignedEndpointUUID) { - this.endpointUUID = endpointUUID; + this.assignedEndpointUUID = assignedEndpointUUID; return this; } @@ -391,7 +391,7 @@ public void mergeFrom(Input input, PlanFragment message) throws IOException break; case 18: - message.endpointUUID = input.readString(); + message.assignedEndpointUUID = input.readString(); break; default: input.handleUnknownField(number, this); @@ -459,8 +459,8 @@ public void writeTo(Output output, PlanFragment message) throws IOException } - if(message.endpointUUID != null) - output.writeString(18, message.endpointUUID, false); + if(message.assignedEndpointUUID != null) + output.writeString(18, message.assignedEndpointUUID, false); } public String getFieldName(int number) @@ -482,7 +482,7 @@ public String getFieldName(int number) case 15: return "optionsJson"; case 16: return "context"; case 17: return "collector"; - case 18: return "endpointUUID"; + case 18: return "assignedEndpointUUID"; default: return null; } } @@ -511,7 +511,7 @@ public int getFieldNumber(String name) __fieldMap.put("optionsJson", 15); __fieldMap.put("context", 16); __fieldMap.put("collector", 17); - __fieldMap.put("endpointUUID", 18); + __fieldMap.put("assignedEndpointUUID", 18); } } diff --git a/protocol/src/main/protobuf/BitControl.proto b/protocol/src/main/protobuf/BitControl.proto index a30b0606017..ae536bce343 100644 --- a/protocol/src/main/protobuf/BitControl.proto +++ b/protocol/src/main/protobuf/BitControl.proto @@ -79,7 +79,7 @@ message PlanFragment { optional string options_json = 15; optional QueryContextInformation context = 16; repeated Collector collector = 17; - optional string endpointUUID = 18; + optional string assignedEndpointUUID = 18; } message Collector { diff --git a/protocol/src/main/protobuf/UserBitShared.proto b/protocol/src/main/protobuf/UserBitShared.proto index 46d28eb4146..090b4cd2e41 100644 --- a/protocol/src/main/protobuf/UserBitShared.proto +++ b/protocol/src/main/protobuf/UserBitShared.proto @@ -268,7 +268,7 @@ message OperatorProfile { optional int64 peak_local_memory_allocated = 7; repeated MetricValue metric = 8; optional int64 wait_nanos = 9; - optional int64 optimal_mem_allocation = 10; + optional int64 max_allocation = 10; } message StreamProfile {