From 0d6ca33ec2d5b6f5be8bd3d78385b620548a7efb Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Fri, 15 May 2015 12:03:59 +0900 Subject: [PATCH 01/21] TAJO-1599 --- .../java/org/apache/tajo/conf/TajoConf.java | 3 +- tajo-common/src/main/proto/tajo_protos.proto | 6 + .../resource/DefaultResourceCalculator.java | 109 ++++++++++ .../apache/tajo/resource/NodeResource.java | 166 +++++++++++++++ .../apache/tajo/resource/NodeResources.java | 191 ++++++++++++++++++ .../tajo/resource/ResourceCalculator.java | 169 ++++++++++++++++ .../tajo/worker/NodeMonitorService.java | 146 +++++++++++++ .../event/NodeResourceAllocateEvent.java | 25 +++ .../event/NodeResourceDeAllocateEvent.java | 25 +++ .../tajo/worker/event/NodeResourceEvent.java | 33 +++ .../apache/tajo/resource/TestResources.java | 48 +++++ .../org/apache/tajo/storage/DiskUtil.java | 4 +- 12 files changed, 922 insertions(+), 3 deletions(-) create mode 100644 tajo-core/src/main/java/org/apache/tajo/resource/DefaultResourceCalculator.java create mode 100644 tajo-core/src/main/java/org/apache/tajo/resource/NodeResource.java create mode 100644 tajo-core/src/main/java/org/apache/tajo/resource/NodeResources.java create mode 100644 tajo-core/src/main/java/org/apache/tajo/resource/ResourceCalculator.java create mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/NodeMonitorService.java create mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceAllocateEvent.java create mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceDeAllocateEvent.java create mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceEvent.java create mode 100644 tajo-core/src/test/java/org/apache/tajo/resource/TestResources.java diff --git a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java index 46e7618a40..07782c1bfa 100644 --- a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java +++ b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java @@ -169,7 +169,8 @@ public static enum ConfVars implements ConfigKey { WORKER_TEMPORAL_DIR_CLEANUP("tajo.worker.tmpdir.cleanup-at-startup", false, Validators.bool()), // Tajo Worker Resources - WORKER_RESOURCE_AVAILABLE_CPU_CORES("tajo.worker.resource.cpu-cores", 1, Validators.min("1")), + WORKER_RESOURCE_AVAILABLE_CPU_CORES("tajo.worker.resource.cpu-cores", + Runtime.getRuntime().availableProcessors(), Validators.min("1")), WORKER_RESOURCE_AVAILABLE_MEMORY_MB("tajo.worker.resource.memory-mb", 1024, Validators.min("64")), WORKER_RESOURCE_AVAILABLE_DISKS("tajo.worker.resource.disks", 1.0f), WORKER_EXECUTION_MAX_SLOTS("tajo.worker.parallel-execution.max-num", 2), diff --git a/tajo-common/src/main/proto/tajo_protos.proto b/tajo-common/src/main/proto/tajo_protos.proto index b6cd9efb6d..8474f546db 100644 --- a/tajo-common/src/main/proto/tajo_protos.proto +++ b/tajo-common/src/main/proto/tajo_protos.proto @@ -62,4 +62,10 @@ message WorkerConnectionInfoProto { optional int32 queryMasterPort = 5; required int32 clientPort = 6; required int32 httpInfoPort = 7; +} + +message NodeResourceProto { + optional int32 memory = 1; + optional int32 virtual_cores = 2; + optional int32 disks = 3; } \ No newline at end of file diff --git a/tajo-core/src/main/java/org/apache/tajo/resource/DefaultResourceCalculator.java b/tajo-core/src/main/java/org/apache/tajo/resource/DefaultResourceCalculator.java new file mode 100644 index 0000000000..58b8a26868 --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/resource/DefaultResourceCalculator.java @@ -0,0 +1,109 @@ +/** +* 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.tajo.resource; + + +public class DefaultResourceCalculator extends ResourceCalculator { + + @Override + public int compare(NodeResource unused, NodeResource lhs, NodeResource rhs) { + return lhs.compareTo(rhs); + } + + @Override + public int computeAvailableContainers(NodeResource available, NodeResource required) { + return Math.min(Math.min( + available.getMemory() / required.getMemory(), + available.getDisks() / required.getDisks()), + available.getVirtualCores() / required.getVirtualCores()); + } + + @Override + public float divide(NodeResource unused, + NodeResource numerator, NodeResource denominator) { + return ratio(numerator, denominator); + } + + public boolean isInvalidDivisor(NodeResource r) { + if (r.getMemory() == 0.0f) { + return true; + } + return false; + } + + @Override + public float ratio(NodeResource a, NodeResource b) { + return (float)a.getMemory() / b.getMemory(); + } + + @Override + public NodeResource divideAndCeil(NodeResource numerator, int denominator) { + return NodeResources.createResource( + divideAndCeil(numerator.getMemory(), denominator)); + } + + @Override + public NodeResource normalize(NodeResource r, NodeResource minimumResource, + NodeResource maximumResource, NodeResource stepFactor) { + int normalizedMemory = Math.min( + roundUp( + Math.max(r.getMemory(), minimumResource.getMemory()), + stepFactor.getMemory()), + maximumResource.getMemory()); + return NodeResources.createResource(normalizedMemory); + } + + @Override + public NodeResource normalize(NodeResource r, NodeResource minimumResource, + NodeResource maximumResource) { + return normalize(r, minimumResource, maximumResource, minimumResource); + } + + @Override + public NodeResource roundUp(NodeResource r, NodeResource stepFactor) { + return NodeResources.createResource( + roundUp(r.getMemory(), stepFactor.getMemory()) + ); + } + + @Override + public NodeResource roundDown(NodeResource r, NodeResource stepFactor) { + return NodeResources.createResource( + roundDown(r.getMemory(), stepFactor.getMemory())); + } + + @Override + public NodeResource multiplyAndNormalizeUp(NodeResource r, double by, + NodeResource stepFactor) { + return NodeResources.createResource( + roundUp((int) (r.getMemory() * by + 0.5), stepFactor.getMemory()) + ); + } + + @Override + public NodeResource multiplyAndNormalizeDown(NodeResource r, double by, + NodeResource stepFactor) { + return NodeResources.createResource( + roundDown( + (int) (r.getMemory() * by), + stepFactor.getMemory() + ) + ); + } + +} diff --git a/tajo-core/src/main/java/org/apache/tajo/resource/NodeResource.java b/tajo-core/src/main/java/org/apache/tajo/resource/NodeResource.java new file mode 100644 index 0000000000..c2512b971c --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/resource/NodeResource.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.tajo.resource; + +import com.google.common.base.Objects; +import org.apache.tajo.TajoProtos; +import org.apache.tajo.common.ProtoObject; + +/** + *

NodeResource models a set of computer resources in the + * cluster.

+ *

+ *

Currently it models memory and disk and CPU.

+ *

+ *

The unit for memory is megabytes. The unit for disks is the number of disk. + * CPU is modeled with virtual cores (vcores), a unit for expressing parallelism. + * A node's capacity should be configured with virtual cores equal to its number of physical cores. + * A task should be requested with the number of cores it can saturate.

+ *

+ */ + +public class NodeResource implements ProtoObject, Comparable { + + private int memory; + private int disks; + private int vCores; + + public NodeResource(TajoProtos.NodeResourceProto proto) { + this.memory = proto.getMemory(); + this.vCores = proto.getVirtualCores(); + this.disks = proto.getDisks(); + } + + private NodeResource() { + + } + + public static NodeResource createResource(int memory, int disks, int vCores) { + return new NodeResource().setMemory(memory).setDisks(disks).setVirtualCores(vCores); + } + + /** + * Get memory of the resource. + * + * @return memory of the resource + */ + public int getMemory() { + return memory; + } + + /** + * Set memory of the resource. + * + * @param memory memory of the resource + */ + public NodeResource setMemory(int memory) { + this.memory = memory; + return this; + } + + + /** + * Get number of disks of the resource. + * + * @return number of disks of the resource + */ + public int getDisks() { + return disks; + } + + /** + * Set number of disks of the resource. + * + * @param disks number of disks of the resource + */ + public NodeResource setDisks(int disks) { + this.disks = disks; + return this; + } + + /** + * Get number of virtual cpu cores of the resource. + * Virtual cores are a unit for expressing CPU parallelism. A node's capacity + * should be configured with virtual cores equal to its number of physical cores. + * + * @return num of virtual cpu cores of the resource + */ + public int getVirtualCores() { + return vCores; + } + + + /** + * Set number of virtual cpu cores of the resource. + * + * @param vCores number of virtual cpu cores of the resource + */ + public NodeResource setVirtualCores(int vCores) { + this.vCores = vCores; + return this; + } + + @Override + public TajoProtos.NodeResourceProto getProto() { + TajoProtos.NodeResourceProto.Builder builder = TajoProtos.NodeResourceProto.newBuilder(); + builder.setMemory(memory) + .setDisks(disks) + .setVirtualCores(vCores); + return builder.build(); + } + + @Override + public int hashCode() { + return Objects.hashCode(getMemory(), getDisks(), getVirtualCores()); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (!(obj instanceof NodeResource)) + return false; + NodeResource other = (NodeResource) obj; + if (getMemory() != other.getMemory() || + getDisks() != other.getDisks() || + getVirtualCores() != other.getVirtualCores()) { + return false; + } + return true; + } + + @Override + public int compareTo(NodeResource other) { + int diff = this.getMemory() - other.getMemory(); + if (diff == 0) { + diff = this.getDisks() - other.getDisks(); + } + if (diff == 0) { + diff = this.getVirtualCores() - other.getVirtualCores(); + } + return diff; + } + + @Override + public String toString() { + return ""; + } +} diff --git a/tajo-core/src/main/java/org/apache/tajo/resource/NodeResources.java b/tajo-core/src/main/java/org/apache/tajo/resource/NodeResources.java new file mode 100644 index 0000000000..51526b6b0a --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/resource/NodeResources.java @@ -0,0 +1,191 @@ +/** +* 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.tajo.resource; + + +public class NodeResources { + + public static NodeResource createResource(int memory) { + return createResource(memory, 0); + } + + public static NodeResource createResource(int memory, int disks) { + return NodeResource.createResource(memory, disks, (memory > 0) ? 1 : 0); + } + + public static NodeResource createResource(int memory, int disks, int vCores) { + return NodeResource.createResource(memory, disks, vCores); + } + + public static NodeResource clone(NodeResource res) { + return NodeResource.createResource(res.getMemory(), res.getVirtualCores(), res.getDisks()); + } + + public static NodeResource addTo(NodeResource lhs, NodeResource rhs) { + lhs.setMemory(lhs.getMemory() + rhs.getMemory()) + .setVirtualCores(lhs.getVirtualCores() + rhs.getVirtualCores()) + .setDisks(lhs.getDisks() + rhs.getDisks()); + return lhs; + } + + public static NodeResource add(NodeResource lhs, NodeResource rhs) { + return addTo(clone(lhs), rhs); + } + + public static NodeResource subtractFrom(NodeResource lhs, NodeResource rhs) { + lhs.setMemory(lhs.getMemory() - rhs.getMemory()) + .setVirtualCores(lhs.getVirtualCores() - rhs.getVirtualCores()) + .setDisks(lhs.getDisks() - rhs.getDisks()); + return lhs; + } + + public static NodeResource subtract(NodeResource lhs, NodeResource rhs) { + return subtractFrom(clone(lhs), rhs); + } + + public static NodeResource multiplyTo(NodeResource lhs, double by) { + lhs.setMemory((int) (lhs.getMemory() * by)) + .setVirtualCores((int) (lhs.getVirtualCores() * by)) + .setDisks((int) (lhs.getDisks() * by)); + return lhs; + } + + public static NodeResource multiply(NodeResource lhs, double by) { + return multiplyTo(clone(lhs), by); + } + + public static NodeResource multiplyAndNormalizeUp( + ResourceCalculator calculator,NodeResource lhs, double by, NodeResource factor) { + return calculator.multiplyAndNormalizeUp(lhs, by, factor); + } + + public static NodeResource multiplyAndNormalizeDown( + ResourceCalculator calculator,NodeResource lhs, double by, NodeResource factor) { + return calculator.multiplyAndNormalizeDown(lhs, by, factor); + } + + public static NodeResource multiplyAndRoundDown(NodeResource lhs, double by) { + NodeResource out = clone(lhs); + out.setMemory((int)(lhs.getMemory() * by)); + out.setDisks((int)(lhs.getDisks() * by)); + out.setVirtualCores((int)(lhs.getVirtualCores() * by)); + return out; + } + + public static NodeResource normalize( + ResourceCalculator calculator, NodeResource lhs, NodeResource min, + NodeResource max, NodeResource increment) { + return calculator.normalize(lhs, min, max, increment); + } + + public static NodeResource roundUp( + ResourceCalculator calculator, NodeResource lhs, NodeResource factor) { + return calculator.roundUp(lhs, factor); + } + + public static NodeResource roundDown( + ResourceCalculator calculator, NodeResource lhs, NodeResource factor) { + return calculator.roundDown(lhs, factor); + } + + public static boolean isInvalidDivisor( + ResourceCalculator resourceCalculator, NodeResource divisor) { + return resourceCalculator.isInvalidDivisor(divisor); + } + + public static float ratio( + ResourceCalculator resourceCalculator, NodeResource lhs, NodeResource rhs) { + return resourceCalculator.ratio(lhs, rhs); + } + + public static float divide( + ResourceCalculator resourceCalculator, + NodeResource clusterResource, NodeResource lhs, NodeResource rhs) { + return resourceCalculator.divide(clusterResource, lhs, rhs); + } + + public static NodeResource divideAndCeil( + ResourceCalculator resourceCalculator, NodeResource lhs, int rhs) { + return resourceCalculator.divideAndCeil(lhs, rhs); + } + + public static boolean equals(NodeResource lhs, NodeResource rhs) { + return lhs.equals(rhs); + } + + public static boolean lessThan( + ResourceCalculator resourceCalculator, + NodeResource clusterResource, + NodeResource lhs, NodeResource rhs) { + return (resourceCalculator.compare(clusterResource, lhs, rhs) < 0); + } + + public static boolean lessThanOrEqual( + ResourceCalculator resourceCalculator, + NodeResource clusterResource, + NodeResource lhs, NodeResource rhs) { + return (resourceCalculator.compare(clusterResource, lhs, rhs) <= 0); + } + + public static boolean greaterThan( + ResourceCalculator resourceCalculator, + NodeResource clusterResource, + NodeResource lhs, NodeResource rhs) { + return resourceCalculator.compare(clusterResource, lhs, rhs) > 0; + } + + public static boolean greaterThanOrEqual( + ResourceCalculator resourceCalculator, + NodeResource clusterResource, + NodeResource lhs, NodeResource rhs) { + return resourceCalculator.compare(clusterResource, lhs, rhs) >= 0; + } + + public static NodeResource min( + ResourceCalculator resourceCalculator, + NodeResource clusterResource, + NodeResource lhs, NodeResource rhs) { + return resourceCalculator.compare(clusterResource, lhs, rhs) <= 0 ? lhs : rhs; + } + + public static NodeResource max( + ResourceCalculator resourceCalculator, + NodeResource clusterResource, + NodeResource lhs, NodeResource rhs) { + return resourceCalculator.compare(clusterResource, lhs, rhs) >= 0 ? lhs : rhs; + } + + public static boolean fitsIn(NodeResource smaller, NodeResource bigger) { + return smaller.getMemory() <= bigger.getMemory() && + smaller.getDisks() <= bigger.getDisks() && + smaller.getVirtualCores() <= bigger.getVirtualCores(); + } + + public static NodeResource componentwiseMin(NodeResource lhs, NodeResource rhs) { + return createResource(Math.min(lhs.getMemory(), rhs.getMemory()), + Math.min(lhs.getDisks(), rhs.getDisks()), + Math.min(lhs.getVirtualCores(), rhs.getVirtualCores())); + } + + public static NodeResource componentwiseMax(NodeResource lhs, NodeResource rhs) { + return createResource(Math.max(lhs.getMemory(), rhs.getMemory()), + Math.max(lhs.getDisks(), rhs.getDisks()), + Math.max(lhs.getVirtualCores(), rhs.getVirtualCores())); + } +} diff --git a/tajo-core/src/main/java/org/apache/tajo/resource/ResourceCalculator.java b/tajo-core/src/main/java/org/apache/tajo/resource/ResourceCalculator.java new file mode 100644 index 0000000000..b08228f026 --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/resource/ResourceCalculator.java @@ -0,0 +1,169 @@ +/** +* 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.tajo.resource; + + +/** + * A set of {@link NodeResource} comparison and manipulation interfaces. + */ + +public abstract class ResourceCalculator { + + public abstract int + compare(NodeResource clusterResource, NodeResource lhs, NodeResource rhs); + + public static int divideAndCeil(int a, int b) { + if (b == 0) { + return 0; + } + return (a + (b - 1)) / b; + } + + public static int roundUp(int a, int b) { + return divideAndCeil(a, b) * b; + } + + public static int roundDown(int a, int b) { + return (a / b) * b; + } + + /** + * Compute the number of containers which can be allocated given + * available and required resources. + * + * @param available available resources + * @param required required resources + * @return number of containers which can be allocated + */ + public abstract int computeAvailableContainers( + NodeResource available, NodeResource required); + /** + * Multiply resource r by factor by + * and normalize up using step-factor stepFactor. + * + * @param r resource to be multiplied + * @param by multiplier + * @param stepFactor factor by which to normalize up + * @return resulting normalized resource + */ + public abstract NodeResource multiplyAndNormalizeUp( + NodeResource r, double by, NodeResource stepFactor); + + /** + * Multiply resource r by factor by + * and normalize down using step-factor stepFactor. + * + * @param r resource to be multiplied + * @param by multiplier + * @param stepFactor factor by which to normalize down + * @return resulting normalized resource + */ + public abstract NodeResource multiplyAndNormalizeDown( + NodeResource r, double by, NodeResource stepFactor); + + /** + * Normalize resource r given the base + * minimumResource and verify against max allowed + * maximumResource + * + * @param r resource + * @param minimumResource step-factor + * @param maximumResource the upper bound of the resource to be allocated + * @return normalized resource + */ + public NodeResource normalize(NodeResource r, NodeResource minimumResource, + NodeResource maximumResource) { + return normalize(r, minimumResource, maximumResource, minimumResource); + } + + /** + * Normalize resource r given the base + * minimumResource and verify against max allowed + * maximumResource using a step factor for hte normalization. + * + * @param r resource + * @param minimumResource minimum value + * @param maximumResource the upper bound of the resource to be allocated + * @param stepFactor the increment for resources to be allocated + * @return normalized resource + */ + public abstract NodeResource normalize(NodeResource r, NodeResource minimumResource, + NodeResource maximumResource, + NodeResource stepFactor); + + + /** + * Round-up resource r given factor stepFactor. + * + * @param r resource + * @param stepFactor step-factor + * @return rounded resource + */ + public abstract NodeResource roundUp(NodeResource r, NodeResource stepFactor); + + /** + * Round-down resource r given factor stepFactor. + * + * @param r resource + * @param stepFactor step-factor + * @return rounded resource + */ + public abstract NodeResource roundDown(NodeResource r, NodeResource stepFactor); + + /** + * Divide resource numerator by resource denominator + * using specified policy (domination, average, fairness etc.); hence overall + * clusterResource is provided for context. + * + * @param clusterResource cluster resources + * @param numerator numerator + * @param denominator denominator + * @return numerator/denominator + * using specific policy + */ + public abstract float divide( + NodeResource clusterResource, NodeResource numerator, NodeResource denominator); + + /** + * Determine if a resource is not suitable for use as a divisor + * (will result in divide by 0, etc) + * + * @param r resource + * @return true if divisor is invalid (should not be used), false else + */ + public abstract boolean isInvalidDivisor(NodeResource r); + + /** + * Ratio of resource a to resource b. + * + * @param a resource + * @param b resource + * @return ratio of resource a to resource b + */ + public abstract float ratio(NodeResource a, NodeResource b); + + /** + * Divide-and-ceil numerator by denominator. + * + * @param numerator numerator resource + * @param denominator denominator + * @return resultant resource + */ + public abstract NodeResource divideAndCeil(NodeResource numerator, int denominator); + +} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/NodeMonitorService.java b/tajo-core/src/main/java/org/apache/tajo/worker/NodeMonitorService.java new file mode 100644 index 0000000000..344f62c81a --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/worker/NodeMonitorService.java @@ -0,0 +1,146 @@ +/** +* Licensed to the Apache Software Foundation (ASF) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. The ASF licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.tajo.worker; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.DataInputByteBuffer; +import org.apache.hadoop.ipc.RPC; +import org.apache.hadoop.security.Credentials; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.util.VersionUtil; +import org.apache.hadoop.yarn.api.records.*; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.event.Dispatcher; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; +import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager; +import org.apache.hadoop.yarn.server.api.ResourceManagerConstants; +import org.apache.hadoop.yarn.server.api.ResourceTracker; +import org.apache.hadoop.yarn.server.api.ServerRMProxy; +import org.apache.hadoop.yarn.server.api.protocolrecords.*; +import org.apache.hadoop.yarn.server.api.records.MasterKey; +import org.apache.hadoop.yarn.server.api.records.NodeAction; +import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus; +import org.apache.hadoop.yarn.server.api.records.NodeStatus; +import org.apache.hadoop.yarn.server.nodemanager.NodeManager.NMContext; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationState; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; +import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics; +import org.apache.hadoop.yarn.server.nodemanager.nodelabels.NodeLabelsProvider; +import org.apache.hadoop.yarn.util.Records; +import org.apache.hadoop.yarn.util.YarnVersionInfo; +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.resource.NodeResource; +import org.apache.tajo.storage.DiskUtil; +import org.apache.tajo.unit.StorageUnit; +import org.apache.tajo.worker.event.NodeResourceEvent; +import org.apache.tajo.worker.event.TaskRunnerEvent; + +import java.io.IOException; +import java.net.ConnectException; +import java.nio.ByteBuffer; +import java.util.*; +import java.util.Map.Entry; +import java.util.concurrent.ConcurrentLinkedQueue; + +public class NodeMonitorService extends AbstractService implements EventHandler { + private static final Log LOG = LogFactory.getLog(NodeMonitorService.class); + + private final Object heartbeatMonitor = new Object(); + private final Dispatcher dispatcher; + private final TajoWorker.WorkerContext context; + private NodeResource totalResource; + private TajoConf tajoConf; + + + private volatile boolean isStopped; + public NodeMonitorService(TajoWorker.WorkerContext context, Dispatcher dispatcher){ + super(NodeMonitorService.class.getName()); + this.dispatcher = dispatcher; + this.context = context; + } + + @Override + protected void serviceInit(Configuration conf) throws Exception { + if (!(conf instanceof TajoConf)) { + throw new IllegalArgumentException("Configuration must be a TajoConf instance"); + } + this.tajoConf = (TajoConf)conf; + this.dispatcher.register(NodeResourceEvent.EventType.class, this); + this.totalResource = createWorkerResource(tajoConf); + + super.serviceInit(conf); + LOG.info("Initialized NodeMonitorService for " + totalResource); + } + + @Override + protected void serviceStart() throws Exception { + + // NodeManager is the last service to start, so NodeId is available. + this.nodeId = this.context.getNodeId(); + this.httpPort = this.context.getHttpPort(); + this.nodeManagerVersionId = YarnVersionInfo.getVersion(); + try { + // Registration has to be in start so that ContainerManager can get the + // perNM tokens needed to authenticate ContainerTokens. + this.resourceTracker = getRMClient(); + registerWithRM(); + super.serviceStart(); + startStatusUpdater(); + } catch (Exception e) { + String errorMessage = "Unexpected error starting NodeStatusUpdater"; + LOG.error(errorMessage, e); + throw new YarnRuntimeException(e); + } + } + + @Override + protected void serviceStop() throws Exception { + // Interrupt the updater. + this.isStopped = true; + super.serviceStop(); + } + + @Override + public void handle(NodeResourceEvent event) { + + } + + private NodeResource createWorkerResource(TajoConf conf) { + + int memoryMb = Math.min((int) (Runtime.getRuntime().maxMemory() / StorageUnit.MB), + conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_MEMORY_MB)); + int vCores = conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES); + int disks = conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_DISKS); + + int dataNodeStorageSize = DiskUtil.getDataNodeStorageSize(); + if (conf.getBoolVar(TajoConf.ConfVars.WORKER_RESOURCE_DFS_DIR_AWARE) && dataNodeStorageSize > 0) { + disks = dataNodeStorageSize; + } + return NodeResource.createResource(memoryMb, disks, vCores); + } +} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceAllocateEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceAllocateEvent.java new file mode 100644 index 0000000000..a3d1d879c2 --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceAllocateEvent.java @@ -0,0 +1,25 @@ +/** + * 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.tajo.worker.event; + +public class NodeResourceAllocateEvent extends NodeResourceEvent { + public NodeResourceAllocateEvent() { + super(EventType.ALLOCATE); + } +} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceDeAllocateEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceDeAllocateEvent.java new file mode 100644 index 0000000000..12dcb7e4de --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceDeAllocateEvent.java @@ -0,0 +1,25 @@ +/** + * 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.tajo.worker.event; + +public class NodeResourceDeallocateEvent extends NodeResourceEvent { + public NodeResourceDeallocateEvent() { + super(EventType.DEALLOCATE); + } +} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceEvent.java new file mode 100644 index 0000000000..a5a349ec46 --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceEvent.java @@ -0,0 +1,33 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.worker.event; + +import org.apache.hadoop.yarn.event.AbstractEvent; +import org.apache.tajo.ExecutionBlockId; + +public class NodeResourceEvent extends AbstractEvent { + public enum EventType { + ALLOCATE, + DEALLOCATE + } + + public NodeResourceEvent(EventType eventType) { + super(eventType); + } +} diff --git a/tajo-core/src/test/java/org/apache/tajo/resource/TestResources.java b/tajo-core/src/test/java/org/apache/tajo/resource/TestResources.java new file mode 100644 index 0000000000..eb0d732dd8 --- /dev/null +++ b/tajo-core/src/test/java/org/apache/tajo/resource/TestResources.java @@ -0,0 +1,48 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.resource; + +import org.junit.Test; + +import static org.apache.tajo.resource.NodeResources.componentwiseMin; +import static org.apache.tajo.resource.NodeResources.createResource; +import static org.apache.tajo.resource.NodeResources.fitsIn; +import static org.junit.Assert.*; + +public class TestResources { + @Test + public void testFitsIn() { + assertTrue(fitsIn(createResource(512, 1, 1), createResource(1024, 2, 1))); + assertTrue(fitsIn(createResource(1024, 2, 1), createResource(1024, 2, 1))); + assertFalse(fitsIn(createResource(1024, 2, 1), createResource(512, 1, 1))); + assertFalse(fitsIn(createResource(512, 2, 1), createResource(1024, 1, 1))); + assertFalse(fitsIn(createResource(1024, 1, 1), createResource(512, 2, 1))); + assertFalse(fitsIn(createResource(512, 1, 2), createResource(512, 1, 1))); + } + + @Test + public void testComponentwiseMin() { + assertEquals(createResource(1, 1), + componentwiseMin(createResource(1, 1), createResource(2, 2))); + assertEquals(createResource(1, 1), + componentwiseMin(createResource(2, 2), createResource(1, 1))); + assertEquals(createResource(1, 1), + componentwiseMin(createResource(1, 2), createResource(2, 1))); + } +} diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/DiskUtil.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/DiskUtil.java index 0bcd5eca3c..19e08e8066 100644 --- a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/DiskUtil.java +++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/DiskUtil.java @@ -189,10 +189,10 @@ private static void setDeviceMountInfo(List deviceInfos) throws } public static int getDataNodeStorageSize(){ - return getStorageDirs().size(); + return getDataNodeStorageDirs().size(); } - public static List getStorageDirs(){ + public static List getDataNodeStorageDirs(){ Configuration conf = new HdfsConfiguration(); Collection dirNames = conf.getTrimmedStringCollection(DFS_DATANODE_DATA_DIR_KEY); return Util.stringCollectionAsURIs(dirNames); From 211f000bb8c14af81b5d77be9ae355b6d0b56cb1 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Mon, 18 May 2015 16:15:14 +0900 Subject: [PATCH 02/21] TAJO-1599 --- .../tajo/master/rm/TajoResourceTracker.java | 7 + .../tajo/worker/NodeMonitorService.java | 146 ---------- .../worker/NodeResourceManagerService.java | 147 ++++++++++ .../tajo/worker/NodeStatusUpdaterService.java | 260 ++++++++++++++++++ .../tajo/worker/WorkerHeartbeatService.java | 1 + .../event/NodeResourceAllocateEvent.java | 25 +- .../event/NodeResourceDeAllocateEvent.java | 14 +- ...ent.java => NodeResourceManagerEvent.java} | 5 +- .../tajo/worker/event/NodeStatusEvent.java | 40 +++ .../main/proto/ResourceTrackerProtocol.proto | 27 ++ .../src/main/proto/TajoWorkerProtocol.proto | 16 ++ 11 files changed, 536 insertions(+), 152 deletions(-) delete mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/NodeMonitorService.java create mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManagerService.java create mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdaterService.java rename tajo-core/src/main/java/org/apache/tajo/worker/event/{NodeResourceEvent.java => NodeResourceManagerEvent.java} (83%) create mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/event/NodeStatusEvent.java diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java index 4f3b66a736..a65c8a546f 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java @@ -182,6 +182,13 @@ public void heartbeat( } } + @Override + public void nodeHeartbeat(RpcController controller, TajoResourceTrackerProtocol.NodeHeartbeatRequestProto request, + RpcCallback done) { + //TODO implement with ResourceManager for scheduler + + } + private Worker createWorkerResource(NodeHeartbeat request) { WorkerResource workerResource = new WorkerResource(); diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/NodeMonitorService.java b/tajo-core/src/main/java/org/apache/tajo/worker/NodeMonitorService.java deleted file mode 100644 index 344f62c81a..0000000000 --- a/tajo-core/src/main/java/org/apache/tajo/worker/NodeMonitorService.java +++ /dev/null @@ -1,146 +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.tajo.worker; - -import com.google.common.annotations.VisibleForTesting; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.classification.InterfaceAudience.Private; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.io.DataInputByteBuffer; -import org.apache.hadoop.ipc.RPC; -import org.apache.hadoop.security.Credentials; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.service.AbstractService; -import org.apache.hadoop.util.StringUtils; -import org.apache.hadoop.util.VersionUtil; -import org.apache.hadoop.yarn.api.records.*; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.event.Dispatcher; -import org.apache.hadoop.yarn.event.EventHandler; -import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; -import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager; -import org.apache.hadoop.yarn.server.api.ResourceManagerConstants; -import org.apache.hadoop.yarn.server.api.ResourceTracker; -import org.apache.hadoop.yarn.server.api.ServerRMProxy; -import org.apache.hadoop.yarn.server.api.protocolrecords.*; -import org.apache.hadoop.yarn.server.api.records.MasterKey; -import org.apache.hadoop.yarn.server.api.records.NodeAction; -import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus; -import org.apache.hadoop.yarn.server.api.records.NodeStatus; -import org.apache.hadoop.yarn.server.nodemanager.NodeManager.NMContext; -import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl; -import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationState; -import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; -import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics; -import org.apache.hadoop.yarn.server.nodemanager.nodelabels.NodeLabelsProvider; -import org.apache.hadoop.yarn.util.Records; -import org.apache.hadoop.yarn.util.YarnVersionInfo; -import org.apache.tajo.conf.TajoConf; -import org.apache.tajo.resource.NodeResource; -import org.apache.tajo.storage.DiskUtil; -import org.apache.tajo.unit.StorageUnit; -import org.apache.tajo.worker.event.NodeResourceEvent; -import org.apache.tajo.worker.event.TaskRunnerEvent; - -import java.io.IOException; -import java.net.ConnectException; -import java.nio.ByteBuffer; -import java.util.*; -import java.util.Map.Entry; -import java.util.concurrent.ConcurrentLinkedQueue; - -public class NodeMonitorService extends AbstractService implements EventHandler { - private static final Log LOG = LogFactory.getLog(NodeMonitorService.class); - - private final Object heartbeatMonitor = new Object(); - private final Dispatcher dispatcher; - private final TajoWorker.WorkerContext context; - private NodeResource totalResource; - private TajoConf tajoConf; - - - private volatile boolean isStopped; - public NodeMonitorService(TajoWorker.WorkerContext context, Dispatcher dispatcher){ - super(NodeMonitorService.class.getName()); - this.dispatcher = dispatcher; - this.context = context; - } - - @Override - protected void serviceInit(Configuration conf) throws Exception { - if (!(conf instanceof TajoConf)) { - throw new IllegalArgumentException("Configuration must be a TajoConf instance"); - } - this.tajoConf = (TajoConf)conf; - this.dispatcher.register(NodeResourceEvent.EventType.class, this); - this.totalResource = createWorkerResource(tajoConf); - - super.serviceInit(conf); - LOG.info("Initialized NodeMonitorService for " + totalResource); - } - - @Override - protected void serviceStart() throws Exception { - - // NodeManager is the last service to start, so NodeId is available. - this.nodeId = this.context.getNodeId(); - this.httpPort = this.context.getHttpPort(); - this.nodeManagerVersionId = YarnVersionInfo.getVersion(); - try { - // Registration has to be in start so that ContainerManager can get the - // perNM tokens needed to authenticate ContainerTokens. - this.resourceTracker = getRMClient(); - registerWithRM(); - super.serviceStart(); - startStatusUpdater(); - } catch (Exception e) { - String errorMessage = "Unexpected error starting NodeStatusUpdater"; - LOG.error(errorMessage, e); - throw new YarnRuntimeException(e); - } - } - - @Override - protected void serviceStop() throws Exception { - // Interrupt the updater. - this.isStopped = true; - super.serviceStop(); - } - - @Override - public void handle(NodeResourceEvent event) { - - } - - private NodeResource createWorkerResource(TajoConf conf) { - - int memoryMb = Math.min((int) (Runtime.getRuntime().maxMemory() / StorageUnit.MB), - conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_MEMORY_MB)); - int vCores = conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES); - int disks = conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_DISKS); - - int dataNodeStorageSize = DiskUtil.getDataNodeStorageSize(); - if (conf.getBoolVar(TajoConf.ConfVars.WORKER_RESOURCE_DFS_DIR_AWARE) && dataNodeStorageSize > 0) { - disks = dataNodeStorageSize; - } - return NodeResource.createResource(memoryMb, disks, vCores); - } -} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManagerService.java b/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManagerService.java new file mode 100644 index 0000000000..7a2a68d2fc --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManagerService.java @@ -0,0 +1,147 @@ +/** +* 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.tajo.worker; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.event.Dispatcher; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.resource.NodeResource; +import org.apache.tajo.resource.NodeResources; +import org.apache.tajo.storage.DiskUtil; +import org.apache.tajo.unit.StorageUnit; +import org.apache.tajo.worker.event.NodeResourceAllocateEvent; +import org.apache.tajo.worker.event.NodeResourceDeallocateEvent; +import org.apache.tajo.worker.event.NodeResourceManagerEvent; + +import java.util.concurrent.atomic.AtomicInteger; + +import static org.apache.tajo.ipc.TajoWorkerProtocol.*; + +public class NodeResourceManagerService extends AbstractService implements EventHandler { + private static final Log LOG = LogFactory.getLog(NodeResourceManagerService.class); + + private final Dispatcher dispatcher; + private final TajoWorker.WorkerContext context; + private NodeResource totalResource; + private NodeResource availableResource; + private AtomicInteger running; + private TajoConf tajoConf; + + private volatile boolean isStopped; + public NodeResourceManagerService(TajoWorker.WorkerContext context, Dispatcher dispatcher){ + super(NodeResourceManagerService.class.getName()); + this.dispatcher = dispatcher; + this.context = context; + } + + @Override + protected void serviceInit(Configuration conf) throws Exception { + if (!(conf instanceof TajoConf)) { + throw new IllegalArgumentException("Configuration must be a TajoConf instance"); + } + this.tajoConf = (TajoConf)conf; + this.totalResource = createWorkerResource(tajoConf); + this.availableResource = NodeResources.clone(totalResource); + this.dispatcher.register(NodeResourceManagerEvent.EventType.class, this); + this.running = new AtomicInteger(); + super.serviceInit(conf); + LOG.info("Initialized NodeMonitorService for " + totalResource); + } + + @Override + protected void serviceStart() throws Exception { + super.serviceStart(); + } + + @Override + protected void serviceStop() throws Exception { + // Interrupt the updater. + this.isStopped = true; + super.serviceStop(); + } + + @Override + public void handle(NodeResourceManagerEvent event) { + switch (event.getType()) { + case ALLOCATE: { + NodeResourceAllocateEvent allocateEvent = (NodeResourceAllocateEvent) event; + BatchAllocationResponseProto.Builder response = BatchAllocationResponseProto.newBuilder(); + for (TaskAllocationRequestProto request : allocateEvent.getRequest().getTaskRequestList()) { + NodeResource resource = new NodeResource(request.getResource()); + if (allocate(resource)) { + running.incrementAndGet(); + //TODO send task event to taskExecutorService + } else { + response.addCancellationTask(request); + } + } + allocateEvent.getCallback().run(response.build()); + break; + } + case DEALLOCATE: { + running.decrementAndGet(); + NodeResourceDeallocateEvent deallocateEvent = (NodeResourceDeallocateEvent) event; + release(deallocateEvent.getResource()); + break; + } + } + } + + public Dispatcher getDispatcher() { + return dispatcher; + } + + public NodeResource getTotalResource() { + return NodeResources.clone(totalResource); + } + + public NodeResource getAvailableResource() { + return NodeResources.clone(availableResource); + } + + private boolean allocate(NodeResource resource) { + if (NodeResources.fitsIn(resource, availableResource)) { + NodeResources.addTo(availableResource, resource); + return true; + } + return false; + } + + private void release(NodeResource resource) { + NodeResources.subtractFrom(availableResource, resource); + } + + private NodeResource createWorkerResource(TajoConf conf) { + + int memoryMb = Math.min((int) (Runtime.getRuntime().maxMemory() / StorageUnit.MB), + conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_MEMORY_MB)); + int vCores = conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES); + int disks = conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_DISKS); + + int dataNodeStorageSize = DiskUtil.getDataNodeStorageSize(); + if (conf.getBoolVar(TajoConf.ConfVars.WORKER_RESOURCE_DFS_DIR_AWARE) && dataNodeStorageSize > 0) { + disks = dataNodeStorageSize; + } + return NodeResource.createResource(memoryMb, disks, vCores); + } +} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdaterService.java b/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdaterService.java new file mode 100644 index 0000000000..5ac100f0d9 --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdaterService.java @@ -0,0 +1,260 @@ +/** + * 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.tajo.worker; + +import com.google.common.collect.Lists; +import com.google.common.collect.Queues; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.ipc.TajoResourceTrackerProtocol; +import org.apache.tajo.resource.NodeResource; +import org.apache.tajo.rpc.*; +import org.apache.tajo.service.ServiceTracker; +import org.apache.tajo.worker.event.NodeStatusEvent; + +import java.net.ConnectException; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicBoolean; + + +import static org.apache.tajo.ipc.TajoResourceTrackerProtocol.*; + +/** + * It periodically sends heartbeat to {@link org.apache.tajo.master.rm.TajoResourceTracker} via asynchronous rpc. + */ +public class NodeStatusUpdaterService extends AbstractService implements EventHandler { + + private final static Log LOG = LogFactory.getLog(NodeStatusUpdaterService.class); + + private TajoConf tajoConf; + private StatusUpdaterThread updaterThread; + private AtomicBoolean flushReport = new AtomicBoolean(); + private volatile boolean isStopped; + private volatile long heartBeatInterval; + private BlockingQueue heartBeatRequestQueue; + private TajoWorker.WorkerContext context; + private NodeResourceManagerService nodeResourceManagerService; + private AsyncRpcClient rmClient; + private TajoResourceTrackerProtocol.TajoResourceTrackerProtocolService resourceTracker; + + public NodeStatusUpdaterService(TajoWorker.WorkerContext context, NodeResourceManagerService resourceManagerService) { + super(NodeStatusUpdaterService.class.getSimpleName()); + this.context = context; + this.nodeResourceManagerService = resourceManagerService; + this.heartBeatInterval = 10 * 1000; //10 sec + } + + @Override + public void serviceInit(Configuration conf) throws Exception { + if (!(conf instanceof TajoConf)) { + throw new IllegalArgumentException("Configuration must be a TajoConf instance"); + } + this.tajoConf = (TajoConf) conf; + this.heartBeatRequestQueue = Queues.newLinkedBlockingQueue(); + this.updaterThread = new StatusUpdaterThread(); + super.serviceInit(conf); + } + + @Override + public void serviceStart() throws Exception { + updaterThread.start(); + super.serviceStart(); + LOG.info("Node status updater started."); + } + + @Override + public void serviceStop() throws Exception { + this.isStopped = true; + + synchronized (updaterThread) { + updaterThread.notifyAll(); + } + super.serviceStop(); + } + + @Override + public void handle(NodeStatusEvent event) { + switch (event.getType()) { + case REPORT_RESOURCE: + heartBeatRequestQueue.add(event); + break; + case FLUSH_REPORTS: + heartBeatRequestQueue.add(event); + synchronized (updaterThread) { + updaterThread.notifyAll(); + } + break; + } + } + + private NodeHeartbeatRequestProto createResourceReport(NodeResource resource) { + NodeHeartbeatRequestProto.Builder requestProto = NodeHeartbeatRequestProto.newBuilder(); + requestProto.setAvailableResource(resource.getProto()); + requestProto.setWorkerId(context.getConnectionInfo().getId()); + return requestProto.build(); + } + + private NodeHeartbeatRequestProto createHeartBeatReport() { + NodeHeartbeatRequestProto.Builder requestProto = NodeHeartbeatRequestProto.newBuilder(); + requestProto.setWorkerId(context.getConnectionInfo().getId()); + return requestProto.build(); + } + + private NodeHeartbeatRequestProto createNodeStatusReport() { + NodeHeartbeatRequestProto.Builder requestProto = NodeHeartbeatRequestProto.newBuilder(); + requestProto.setTotalResource(nodeResourceManagerService.getTotalResource().getProto()); + requestProto.setAvailableResource(nodeResourceManagerService.getAvailableResource().getProto()); + requestProto.setWorkerId(context.getConnectionInfo().getId()); + requestProto.setConnectionInfo(context.getConnectionInfo().getProto()); + + //TODO set node status to requestProto.setStatus() + return requestProto.build(); + } + + private TajoResourceTrackerProtocol.TajoResourceTrackerProtocolService newStub() + throws NoSuchMethodException, ConnectException, ClassNotFoundException { + ServiceTracker serviceTracker = context.getServiceTracker(); + RpcClientManager.cleanup(rmClient); + + RpcClientManager rpcManager = RpcClientManager.getInstance(); + rmClient = rpcManager.newClient(serviceTracker.getResourceTrackerAddress(), + TajoResourceTrackerProtocol.class, true, rpcManager.getRetries(), + rpcManager.getTimeoutSeconds(), TimeUnit.SECONDS, false); + return rmClient.getStub(); + } + + protected NodeHeartbeatResponseProto sendHeartbeat(NodeHeartbeatRequestProto requestProto) + throws NoSuchMethodException, ClassNotFoundException, ConnectException, ExecutionException { + if (resourceTracker == null) { + resourceTracker = newStub(); + } + + NodeHeartbeatResponseProto response = null; + try { + CallFuture callBack = new CallFuture(); + + resourceTracker.nodeHeartbeat(callBack.getController(), requestProto, callBack); + response = callBack.get(RpcConstants.DEFAULT_FUTURE_TIMEOUT_SECONDS, TimeUnit.SECONDS); + } catch (InterruptedException e) { + LOG.warn(e.getMessage()); + } catch (TimeoutException te) { + LOG.warn("Heartbeat response is being delayed.", te); + } catch (ExecutionException ee) { + LOG.warn("TajoMaster failure: " + ee.getMessage()); + resourceTracker = null; + throw ee; + } + return response; + } + + class StatusUpdaterThread extends Thread { + + public StatusUpdaterThread() { + super("NodeStatusUpdater"); + } + + private int drain(Collection buffer, int numElements, + long timeout, TimeUnit unit) throws InterruptedException { + + long deadline = System.nanoTime() + unit.toNanos(timeout); + int added = 0; + while (added < numElements) { + added += heartBeatRequestQueue.drainTo(buffer, numElements - added); + if (added < numElements) { // not enough elements immediately available; will have to wait + if (deadline <= System.currentTimeMillis()) { + break; + } else { + synchronized (updaterThread) { + updaterThread.wait(deadline - System.currentTimeMillis()); + if (deadline > System.nanoTime()) { + added += heartBeatRequestQueue.drainTo(buffer, numElements - added); + break; + } + } + } + } + } + return added; + } + + @Override + public void run() { + NodeHeartbeatResponseProto lastResponse = null; + while (!isStopped && !Thread.interrupted()) { + + try { + if (lastResponse != null) { + if (lastResponse.getCommand() == ResponseCommand.NORMAL) { + List events = Lists.newArrayList(); + try { + drain(events, + Math.max(nodeResourceManagerService.getTotalResource().getVirtualCores() / 2, 1), + heartBeatInterval, TimeUnit.MILLISECONDS); + } catch (InterruptedException e) { + break; + } + + if (!events.isEmpty()) { + // send last available resource; + lastResponse = sendHeartbeat(createResourceReport(events.get(events.size() - 1).getResource())); + } else { + lastResponse = sendHeartbeat(createHeartBeatReport()); + } + + } else if (lastResponse.getCommand() == ResponseCommand.MEMBERSHIP) { + lastResponse = sendHeartbeat(createNodeStatusReport()); + } else if (lastResponse.getCommand() == ResponseCommand.ABORT_QUERY) { + //TODO abort failure queries + } + } else { + lastResponse = sendHeartbeat(createNodeStatusReport()); + } + } catch (NoSuchMethodException e) { + LOG.fatal(e.getMessage(), e); + Runtime.getRuntime().halt(1); + } catch (ClassNotFoundException e) { + LOG.fatal(e.getMessage(), e); + Runtime.getRuntime().halt(1); + } catch (Exception e) { + LOG.error(e.getMessage(), e); + if (!isStopped) { + synchronized (updaterThread) { + try { + updaterThread.wait(heartBeatInterval); + } catch (InterruptedException e1) { + // Do Nothing + } + } + } + } + } + + LOG.info("Heartbeat Thread stopped."); + } + } +} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/WorkerHeartbeatService.java b/tajo-core/src/main/java/org/apache/tajo/worker/WorkerHeartbeatService.java index 9afee5a003..2c43049b78 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/WorkerHeartbeatService.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/WorkerHeartbeatService.java @@ -47,6 +47,7 @@ /** * It periodically sends heartbeat to {@link org.apache.tajo.master.rm.TajoResourceTracker} via asynchronous rpc. */ +@Deprecated public class WorkerHeartbeatService extends AbstractService { /** class logger */ private final static Log LOG = LogFactory.getLog(WorkerHeartbeatService.class); diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceAllocateEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceAllocateEvent.java index a3d1d879c2..2f411e8c5c 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceAllocateEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceAllocateEvent.java @@ -18,8 +18,29 @@ package org.apache.tajo.worker.event; -public class NodeResourceAllocateEvent extends NodeResourceEvent { - public NodeResourceAllocateEvent() { + +import com.google.protobuf.RpcCallback; + +import static org.apache.tajo.ipc.TajoWorkerProtocol.BatchAllocationRequestProto; +import static org.apache.tajo.ipc.TajoWorkerProtocol.BatchAllocationResponseProto; + +public class NodeResourceAllocateEvent extends NodeResourceManagerEvent { + + private BatchAllocationRequestProto request; + private RpcCallback callback; + + public NodeResourceAllocateEvent(BatchAllocationRequestProto request, + RpcCallback callback) { super(EventType.ALLOCATE); + this.callback = callback; + this.request = request; + } + + public BatchAllocationRequestProto getRequest() { + return request; + } + + public RpcCallback getCallback() { + return callback; } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceDeAllocateEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceDeAllocateEvent.java index 12dcb7e4de..7011101b05 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceDeAllocateEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceDeAllocateEvent.java @@ -18,8 +18,18 @@ package org.apache.tajo.worker.event; -public class NodeResourceDeallocateEvent extends NodeResourceEvent { - public NodeResourceDeallocateEvent() { +import org.apache.tajo.resource.NodeResource; + +public class NodeResourceDeallocateEvent extends NodeResourceManagerEvent { + + private NodeResource resource; + + public NodeResourceDeallocateEvent(NodeResource resource) { super(EventType.DEALLOCATE); + this.resource = resource; + } + + public NodeResource getResource() { + return resource; } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceManagerEvent.java similarity index 83% rename from tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceEvent.java rename to tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceManagerEvent.java index a5a349ec46..bcb34486c0 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceManagerEvent.java @@ -20,14 +20,15 @@ import org.apache.hadoop.yarn.event.AbstractEvent; import org.apache.tajo.ExecutionBlockId; +import org.apache.tajo.resource.NodeResource; -public class NodeResourceEvent extends AbstractEvent { +public class NodeResourceManagerEvent extends AbstractEvent { public enum EventType { ALLOCATE, DEALLOCATE } - public NodeResourceEvent(EventType eventType) { + public NodeResourceManagerEvent(EventType eventType) { super(eventType); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeStatusEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeStatusEvent.java new file mode 100644 index 0000000000..58ab74af0a --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeStatusEvent.java @@ -0,0 +1,40 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.worker.event; + +import org.apache.hadoop.yarn.event.AbstractEvent; +import org.apache.tajo.resource.NodeResource; + +public class NodeStatusEvent extends AbstractEvent { + private final NodeResource resource; + + public enum EventType { + REPORT_RESOURCE, + FLUSH_REPORTS + } + + public NodeStatusEvent(EventType eventType, NodeResource resource) { + super(eventType); + this.resource = resource; + } + + public NodeResource getResource() { + return resource; + } +} diff --git a/tajo-core/src/main/proto/ResourceTrackerProtocol.proto b/tajo-core/src/main/proto/ResourceTrackerProtocol.proto index 40aeab7360..dffd8c9ee4 100644 --- a/tajo-core/src/main/proto/ResourceTrackerProtocol.proto +++ b/tajo-core/src/main/proto/ResourceTrackerProtocol.proto @@ -25,15 +25,42 @@ option java_generate_equals_and_hash = true; import "QueryCoordinatorProtocol.proto"; import "ContainerProtocol.proto"; import "tajo_protos.proto"; +import "TajoIdProtos.proto"; package hadoop.yarn; +// deprecated message NodeHeartbeat { required WorkerConnectionInfoProto connectionInfo = 1; optional ServerStatusProto serverStatus = 2; optional string statusMessage = 3; } +message NodeHeartbeatRequestProto { + required int32 workerId = 1; + optional NodeResourceProto totalResource = 2; + optional NodeResourceProto availableResource = 3; + optional WorkerConnectionInfoProto connectionInfo = 4; + optional NodeStatusProto status = 5; +} + +message NodeHeartbeatResponseProto { + required ResponseCommand command = 1 [default = NORMAL]; + repeated QueryIdProto queryId = 2; +} + +enum ResponseCommand { + NORMAL = 1; //ping + MEMBERSHIP = 2; // request membership to worker node + ABORT_QUERY = 3; //query master failure + SHUTDOWN = 4; // black list +} + +//TODO add node health information +message NodeStatusProto { +} + service TajoResourceTrackerProtocolService { rpc heartbeat(NodeHeartbeat) returns (TajoHeartbeatResponse); + rpc nodeHeartbeat(NodeHeartbeatRequestProto) returns (NodeHeartbeatResponseProto); } \ No newline at end of file diff --git a/tajo-core/src/main/proto/TajoWorkerProtocol.proto b/tajo-core/src/main/proto/TajoWorkerProtocol.proto index bf9bbde509..2324596b1b 100644 --- a/tajo-core/src/main/proto/TajoWorkerProtocol.proto +++ b/tajo-core/src/main/proto/TajoWorkerProtocol.proto @@ -116,6 +116,7 @@ message ExecutionBlockReport { repeated IntermediateEntryProto intermediateEntries = 5; } +// deprecated message TaskResponseProto { required string id = 1; required QueryState status = 2; @@ -161,6 +162,7 @@ message QueryExecutionRequestProto { optional StringProto logicalPlanJson = 6; } +// deprecated message GetTaskRequestProto { required int32 workerId = 1; required TajoContainerIdProto containerId = 2; @@ -198,6 +200,20 @@ message ExecutionBlockListProto { repeated ExecutionBlockIdProto executionBlockId = 1; } +message TaskAllocationRequestProto { + required TaskRequestProto taskRequest = 1; + required NodeResourceProto resource = 2; +} + +message BatchAllocationRequestProto { + required ExecutionBlockIdProto executionBlockId = 1; + repeated TaskAllocationRequestProto taskRequest = 2; +} + +message BatchAllocationResponseProto { + repeated TaskAllocationRequestProto cancellationTask = 2; +} + service TajoWorkerProtocolService { rpc ping (TaskAttemptIdProto) returns (BoolProto); From c0257fdc33009f26e31cbd8a1043f3bd6f7600ce Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Tue, 19 May 2015 01:42:49 +0900 Subject: [PATCH 03/21] add test cases --- .../java/org/apache/tajo/conf/TajoConf.java | 4 +- .../tajo/master/rm/TajoResourceTracker.java | 4 +- .../apache/tajo/resource/NodeResources.java | 6 +- .../worker/NodeResourceManagerService.java | 60 ++--- .../tajo/worker/NodeStatusUpdaterService.java | 92 ++++--- .../event/NodeResourceDeAllocateEvent.java | 4 + .../worker/MockNodeStatusUpdaterService.java | 105 ++++++++ .../TestNodeResourceManagerService.java | 242 ++++++++++++++++++ .../worker/TestNodeStatusUpdaterService.java | 115 +++++++++ 9 files changed, 561 insertions(+), 71 deletions(-) create mode 100644 tajo-core/src/test/java/org/apache/tajo/worker/MockNodeStatusUpdaterService.java create mode 100644 tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManagerService.java create mode 100644 tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdaterService.java diff --git a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java index d107eb8b9c..995b8d639c 100644 --- a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java +++ b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java @@ -174,7 +174,9 @@ public static enum ConfVars implements ConfigKey { WORKER_RESOURCE_AVAILABLE_CPU_CORES("tajo.worker.resource.cpu-cores", Runtime.getRuntime().availableProcessors(), Validators.min("1")), WORKER_RESOURCE_AVAILABLE_MEMORY_MB("tajo.worker.resource.memory-mb", 1024, Validators.min("64")), + @Deprecated WORKER_RESOURCE_AVAILABLE_DISKS("tajo.worker.resource.disks", 1.0f), + WORKER_RESOURCE_AVAILABLE_DISK_NUM("tajo.worker.resource.disk.num", 1), WORKER_EXECUTION_MAX_SLOTS("tajo.worker.parallel-execution.max-num", 2), WORKER_RESOURCE_DFS_DIR_AWARE("tajo.worker.resource.dfs-dir-aware", false, Validators.bool()), @@ -187,7 +189,7 @@ public static enum ConfVars implements ConfigKey { WORKER_HISTORY_EXPIRE_PERIOD("tajo.worker.history.expire-interval-minutes", 60), // 1 hours QUERYMASTER_HISTORY_EXPIRE_PERIOD("tajo.qm.history.expire-interval-minutes", 6 * 60), // 6 hours - WORKER_HEARTBEAT_TIMEOUT("tajo.worker.heartbeat.timeout", 120 * 1000), // 120 sec + WORKER_HEARTBEAT_INTERVAL("tajo.worker.heartbeat.interval", 10 * 1000), // 10 sec // Resource Manager RESOURCE_MANAGER_CLASS("tajo.resource.manager", "org.apache.tajo.master.rm.TajoWorkerResourceManager", diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java index a65c8a546f..af28886446 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java @@ -20,10 +20,12 @@ import com.google.protobuf.RpcCallback; import com.google.protobuf.RpcController; +import com.google.protobuf.ServiceException; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.service.AbstractService; +import org.apache.tajo.common.exception.NotImplementedException; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.ipc.QueryCoordinatorProtocol.TajoHeartbeatResponse; import org.apache.tajo.ipc.TajoResourceTrackerProtocol; @@ -186,7 +188,7 @@ public void heartbeat( public void nodeHeartbeat(RpcController controller, TajoResourceTrackerProtocol.NodeHeartbeatRequestProto request, RpcCallback done) { //TODO implement with ResourceManager for scheduler - + throw new RuntimeException(new ServiceException(new NotImplementedException().getMessage())); } private Worker createWorkerResource(NodeHeartbeat request) { diff --git a/tajo-core/src/main/java/org/apache/tajo/resource/NodeResources.java b/tajo-core/src/main/java/org/apache/tajo/resource/NodeResources.java index 51526b6b0a..01e9dcf5bc 100644 --- a/tajo-core/src/main/java/org/apache/tajo/resource/NodeResources.java +++ b/tajo-core/src/main/java/org/apache/tajo/resource/NodeResources.java @@ -34,7 +34,11 @@ public static NodeResource createResource(int memory, int disks, int vCores) { } public static NodeResource clone(NodeResource res) { - return NodeResource.createResource(res.getMemory(), res.getVirtualCores(), res.getDisks()); + return NodeResource.createResource(res.getMemory(), res.getDisks(), res.getVirtualCores()); + } + + public static NodeResource update(NodeResource lhs, NodeResource rhs) { + return lhs.setMemory(rhs.getMemory()).setDisks(rhs.getDisks()).setVirtualCores(rhs.getVirtualCores()); } public static NodeResource addTo(NodeResource lhs, NodeResource rhs) { diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManagerService.java b/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManagerService.java index 7a2a68d2fc..c4c57cb628 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManagerService.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManagerService.java @@ -29,9 +29,11 @@ import org.apache.tajo.resource.NodeResources; import org.apache.tajo.storage.DiskUtil; import org.apache.tajo.unit.StorageUnit; +import org.apache.tajo.util.CommonTestingUtil; import org.apache.tajo.worker.event.NodeResourceAllocateEvent; import org.apache.tajo.worker.event.NodeResourceDeallocateEvent; import org.apache.tajo.worker.event.NodeResourceManagerEvent; +import org.apache.tajo.worker.event.NodeStatusEvent; import java.util.concurrent.atomic.AtomicInteger; @@ -41,17 +43,14 @@ public class NodeResourceManagerService extends AbstractService implements Event private static final Log LOG = LogFactory.getLog(NodeResourceManagerService.class); private final Dispatcher dispatcher; - private final TajoWorker.WorkerContext context; private NodeResource totalResource; private NodeResource availableResource; - private AtomicInteger running; + private AtomicInteger allocatedSize; private TajoConf tajoConf; - private volatile boolean isStopped; - public NodeResourceManagerService(TajoWorker.WorkerContext context, Dispatcher dispatcher){ + public NodeResourceManagerService(Dispatcher dispatcher){ super(NodeResourceManagerService.class.getName()); this.dispatcher = dispatcher; - this.context = context; } @Override @@ -63,21 +62,9 @@ protected void serviceInit(Configuration conf) throws Exception { this.totalResource = createWorkerResource(tajoConf); this.availableResource = NodeResources.clone(totalResource); this.dispatcher.register(NodeResourceManagerEvent.EventType.class, this); - this.running = new AtomicInteger(); + this.allocatedSize = new AtomicInteger(); super.serviceInit(conf); - LOG.info("Initialized NodeMonitorService for " + totalResource); - } - - @Override - protected void serviceStart() throws Exception { - super.serviceStart(); - } - - @Override - protected void serviceStop() throws Exception { - // Interrupt the updater. - this.isStopped = true; - super.serviceStop(); + LOG.info("Initialized NodeResourceManagerService for " + totalResource); } @Override @@ -89,7 +76,7 @@ public void handle(NodeResourceManagerEvent event) { for (TaskAllocationRequestProto request : allocateEvent.getRequest().getTaskRequestList()) { NodeResource resource = new NodeResource(request.getResource()); if (allocate(resource)) { - running.incrementAndGet(); + allocatedSize.incrementAndGet(); //TODO send task event to taskExecutorService } else { response.addCancellationTask(request); @@ -99,44 +86,59 @@ public void handle(NodeResourceManagerEvent event) { break; } case DEALLOCATE: { - running.decrementAndGet(); + allocatedSize.decrementAndGet(); NodeResourceDeallocateEvent deallocateEvent = (NodeResourceDeallocateEvent) event; release(deallocateEvent.getResource()); + + // send current resource to ResourceTracker + getDispatcher().getEventHandler().handle( + new NodeStatusEvent(NodeStatusEvent.EventType.REPORT_RESOURCE, getAvailableResource())); break; } } } - public Dispatcher getDispatcher() { + protected Dispatcher getDispatcher() { return dispatcher; } - public NodeResource getTotalResource() { + protected NodeResource getTotalResource() { return NodeResources.clone(totalResource); } - public NodeResource getAvailableResource() { + protected NodeResource getAvailableResource() { return NodeResources.clone(availableResource); } + public int getAllocatedSize() { + return allocatedSize.get(); + } + private boolean allocate(NodeResource resource) { + //TODO consider the jvm free memory if (NodeResources.fitsIn(resource, availableResource)) { - NodeResources.addTo(availableResource, resource); + NodeResources.subtractFrom(availableResource, resource); return true; } return false; } private void release(NodeResource resource) { - NodeResources.subtractFrom(availableResource, resource); + NodeResources.addTo(availableResource, resource); } private NodeResource createWorkerResource(TajoConf conf) { + int memoryMb; + + if (conf.get(CommonTestingUtil.TAJO_TEST_KEY, "FALSE").equalsIgnoreCase("TRUE")) { + memoryMb = conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_MEMORY_MB); + } else { + memoryMb = Math.min((int) (Runtime.getRuntime().maxMemory() / StorageUnit.MB), + conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_MEMORY_MB)); + } - int memoryMb = Math.min((int) (Runtime.getRuntime().maxMemory() / StorageUnit.MB), - conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_MEMORY_MB)); int vCores = conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES); - int disks = conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_DISKS); + int disks = conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_DISK_NUM); int dataNodeStorageSize = DiskUtil.getDataNodeStorageSize(); if (conf.getBoolVar(TajoConf.ConfVars.WORKER_RESOURCE_DFS_DIR_AWARE) && dataNodeStorageSize > 0) { diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdaterService.java b/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdaterService.java index 5ac100f0d9..b16b46174c 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdaterService.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdaterService.java @@ -27,9 +27,11 @@ import org.apache.hadoop.yarn.event.EventHandler; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.ipc.TajoResourceTrackerProtocol; +import org.apache.tajo.master.cluster.WorkerConnectionInfo; import org.apache.tajo.resource.NodeResource; import org.apache.tajo.rpc.*; import org.apache.tajo.service.ServiceTracker; +import org.apache.tajo.service.ServiceTrackerFactory; import org.apache.tajo.worker.event.NodeStatusEvent; import java.net.ConnectException; @@ -39,7 +41,6 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import java.util.concurrent.atomic.AtomicBoolean; import static org.apache.tajo.ipc.TajoResourceTrackerProtocol.*; @@ -53,20 +54,20 @@ public class NodeStatusUpdaterService extends AbstractService implements EventHa private TajoConf tajoConf; private StatusUpdaterThread updaterThread; - private AtomicBoolean flushReport = new AtomicBoolean(); private volatile boolean isStopped; private volatile long heartBeatInterval; - private BlockingQueue heartBeatRequestQueue; - private TajoWorker.WorkerContext context; - private NodeResourceManagerService nodeResourceManagerService; + private BlockingQueue heartBeatRequestQueue; + private final WorkerConnectionInfo connectionInfo; + private final NodeResourceManagerService nodeResourceManagerService; private AsyncRpcClient rmClient; - private TajoResourceTrackerProtocol.TajoResourceTrackerProtocolService resourceTracker; + private ServiceTracker serviceTracker; + private TajoResourceTrackerProtocolService.Interface resourceTracker; + private int queueingLimit; - public NodeStatusUpdaterService(TajoWorker.WorkerContext context, NodeResourceManagerService resourceManagerService) { + public NodeStatusUpdaterService(WorkerConnectionInfo connectionInfo, NodeResourceManagerService resourceManagerService) { super(NodeStatusUpdaterService.class.getSimpleName()); - this.context = context; + this.connectionInfo = connectionInfo; this.nodeResourceManagerService = resourceManagerService; - this.heartBeatInterval = 10 * 1000; //10 sec } @Override @@ -76,15 +77,21 @@ public void serviceInit(Configuration conf) throws Exception { } this.tajoConf = (TajoConf) conf; this.heartBeatRequestQueue = Queues.newLinkedBlockingQueue(); + this.serviceTracker = ServiceTrackerFactory.get(tajoConf); + this.nodeResourceManagerService.getDispatcher().register(NodeStatusEvent.EventType.class, this); + this.heartBeatInterval = tajoConf.getIntVar(TajoConf.ConfVars.WORKER_HEARTBEAT_INTERVAL); this.updaterThread = new StatusUpdaterThread(); super.serviceInit(conf); } @Override public void serviceStart() throws Exception { + // if resource changed over than 50%, send reports + this.queueingLimit = nodeResourceManagerService.getTotalResource().getVirtualCores() / 2; + updaterThread.start(); super.serviceStart(); - LOG.info("Node status updater started."); + LOG.info("NodeStatusUpdater started."); } @Override @@ -95,33 +102,39 @@ public void serviceStop() throws Exception { updaterThread.notifyAll(); } super.serviceStop(); + LOG.info("NodeStatusUpdater stopped."); } @Override public void handle(NodeStatusEvent event) { switch (event.getType()) { case REPORT_RESOURCE: - heartBeatRequestQueue.add(event); + heartBeatRequestQueue.add(event); //batch report to ResourceTracker break; case FLUSH_REPORTS: - heartBeatRequestQueue.add(event); - synchronized (updaterThread) { - updaterThread.notifyAll(); - } + heartBeatRequestQueue.add(event); //flush report to ResourceTracker break; } } + public int getQueueSize() { + return heartBeatRequestQueue.size(); + } + + public int getQueueingLimit() { + return queueingLimit; + } + private NodeHeartbeatRequestProto createResourceReport(NodeResource resource) { NodeHeartbeatRequestProto.Builder requestProto = NodeHeartbeatRequestProto.newBuilder(); requestProto.setAvailableResource(resource.getProto()); - requestProto.setWorkerId(context.getConnectionInfo().getId()); + requestProto.setWorkerId(connectionInfo.getId()); return requestProto.build(); } private NodeHeartbeatRequestProto createHeartBeatReport() { NodeHeartbeatRequestProto.Builder requestProto = NodeHeartbeatRequestProto.newBuilder(); - requestProto.setWorkerId(context.getConnectionInfo().getId()); + requestProto.setWorkerId(connectionInfo.getId()); return requestProto.build(); } @@ -129,16 +142,15 @@ private NodeHeartbeatRequestProto createNodeStatusReport() { NodeHeartbeatRequestProto.Builder requestProto = NodeHeartbeatRequestProto.newBuilder(); requestProto.setTotalResource(nodeResourceManagerService.getTotalResource().getProto()); requestProto.setAvailableResource(nodeResourceManagerService.getAvailableResource().getProto()); - requestProto.setWorkerId(context.getConnectionInfo().getId()); - requestProto.setConnectionInfo(context.getConnectionInfo().getProto()); + requestProto.setWorkerId(connectionInfo.getId()); + requestProto.setConnectionInfo(connectionInfo.getProto()); //TODO set node status to requestProto.setStatus() return requestProto.build(); } - private TajoResourceTrackerProtocol.TajoResourceTrackerProtocolService newStub() + protected TajoResourceTrackerProtocolService.Interface newStub() throws NoSuchMethodException, ConnectException, ClassNotFoundException { - ServiceTracker serviceTracker = context.getServiceTracker(); RpcClientManager.cleanup(rmClient); RpcClientManager rpcManager = RpcClientManager.getInstance(); @@ -178,7 +190,7 @@ public StatusUpdaterThread() { super("NodeStatusUpdater"); } - private int drain(Collection buffer, int numElements, + private int drain(Collection buffer, int numElements, long timeout, TimeUnit unit) throws InterruptedException { long deadline = System.nanoTime() + unit.toNanos(timeout); @@ -186,22 +198,22 @@ private int drain(Collection buffer, int numElements, while (added < numElements) { added += heartBeatRequestQueue.drainTo(buffer, numElements - added); if (added < numElements) { // not enough elements immediately available; will have to wait - if (deadline <= System.currentTimeMillis()) { + NodeStatusEvent e = heartBeatRequestQueue.poll(deadline - System.nanoTime(), TimeUnit.NANOSECONDS); + if (e == null) { + break; // we already waited enough, and there are no more elements in sight + } + buffer.add(e); + added++; + + if (e.getType() == NodeStatusEvent.EventType.FLUSH_REPORTS) { break; - } else { - synchronized (updaterThread) { - updaterThread.wait(deadline - System.currentTimeMillis()); - if (deadline > System.nanoTime()) { - added += heartBeatRequestQueue.drainTo(buffer, numElements - added); - break; - } - } } } } return added; } + /* Node sends a heartbeats with its resource and status periodically to master. */ @Override public void run() { NodeHeartbeatResponseProto lastResponse = null; @@ -212,9 +224,8 @@ public void run() { if (lastResponse.getCommand() == ResponseCommand.NORMAL) { List events = Lists.newArrayList(); try { - drain(events, - Math.max(nodeResourceManagerService.getTotalResource().getVirtualCores() / 2, 1), - heartBeatInterval, TimeUnit.MILLISECONDS); + /* batch update to ResourceTracker */ + drain(events, Math.max(queueingLimit, 1), heartBeatInterval, TimeUnit.MILLISECONDS); } catch (InterruptedException e) { break; } @@ -223,22 +234,25 @@ public void run() { // send last available resource; lastResponse = sendHeartbeat(createResourceReport(events.get(events.size() - 1).getResource())); } else { + // send ping; lastResponse = sendHeartbeat(createHeartBeatReport()); } } else if (lastResponse.getCommand() == ResponseCommand.MEMBERSHIP) { + // Membership changed lastResponse = sendHeartbeat(createNodeStatusReport()); } else if (lastResponse.getCommand() == ResponseCommand.ABORT_QUERY) { //TODO abort failure queries } } else { + // Node registration on startup lastResponse = sendHeartbeat(createNodeStatusReport()); } - } catch (NoSuchMethodException e) { - LOG.fatal(e.getMessage(), e); + } catch (NoSuchMethodException nsme) { + LOG.fatal(nsme.getMessage(), nsme); Runtime.getRuntime().halt(1); - } catch (ClassNotFoundException e) { - LOG.fatal(e.getMessage(), e); + } catch (ClassNotFoundException cnfe) { + LOG.fatal(cnfe.getMessage(), cnfe); Runtime.getRuntime().halt(1); } catch (Exception e) { LOG.error(e.getMessage(), e); @@ -246,7 +260,7 @@ public void run() { synchronized (updaterThread) { try { updaterThread.wait(heartBeatInterval); - } catch (InterruptedException e1) { + } catch (InterruptedException ie) { // Do Nothing } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceDeAllocateEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceDeAllocateEvent.java index 7011101b05..78a7796b33 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceDeAllocateEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceDeAllocateEvent.java @@ -18,12 +18,16 @@ package org.apache.tajo.worker.event; +import org.apache.tajo.TajoProtos; import org.apache.tajo.resource.NodeResource; public class NodeResourceDeallocateEvent extends NodeResourceManagerEvent { private NodeResource resource; + public NodeResourceDeallocateEvent(TajoProtos.NodeResourceProto proto) { + this(new NodeResource(proto)); + } public NodeResourceDeallocateEvent(NodeResource resource) { super(EventType.DEALLOCATE); this.resource = resource; diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeStatusUpdaterService.java b/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeStatusUpdaterService.java new file mode 100644 index 0000000000..934f29259c --- /dev/null +++ b/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeStatusUpdaterService.java @@ -0,0 +1,105 @@ +/** + * 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.tajo.worker; + +import com.google.common.collect.Maps; +import com.google.protobuf.RpcCallback; +import com.google.protobuf.RpcController; +import org.apache.tajo.ipc.QueryCoordinatorProtocol; +import org.apache.tajo.master.cluster.WorkerConnectionInfo; +import org.apache.tajo.resource.NodeResource; +import org.apache.tajo.resource.NodeResources; + +import java.net.ConnectException; +import java.util.Map; +import java.util.concurrent.CountDownLatch; + +import static org.apache.tajo.ipc.TajoResourceTrackerProtocol.*; + +public class MockNodeStatusUpdaterService extends NodeStatusUpdaterService { + + private CountDownLatch barrier; + private Map membership = Maps.newHashMap(); + private Map resources = Maps.newHashMap(); + private MockResourceTracker resourceTracker; + + public MockNodeStatusUpdaterService(CountDownLatch barrier, WorkerConnectionInfo connectionInfo, + NodeResourceManagerService resourceManagerService) { + super(connectionInfo, resourceManagerService); + this.barrier = barrier; + this.resourceTracker = new MockResourceTracker(); + } + + @Override + protected TajoResourceTrackerProtocolService.Interface newStub() + throws NoSuchMethodException, ConnectException, ClassNotFoundException { + + return resourceTracker; + } + + protected MockResourceTracker getResourceTracker() { + return resourceTracker; + } + + class MockResourceTracker implements TajoResourceTrackerProtocolService.Interface { + private NodeHeartbeatRequestProto lastRequest; + + protected Map getTotalResource() { + return membership; + } + + protected Map getAvailableResource() { + return membership; + } + + protected NodeHeartbeatRequestProto getLastRequest() { + return lastRequest; + } + + @Override + public void heartbeat(RpcController controller, NodeHeartbeat request, + RpcCallback done) { + + } + + @Override + public void nodeHeartbeat(RpcController controller, NodeHeartbeatRequestProto request, + RpcCallback done) { + + NodeHeartbeatResponseProto.Builder response = NodeHeartbeatResponseProto.newBuilder(); + if (membership.containsKey(request.getWorkerId())) { + if (request.hasAvailableResource()) { + NodeResource resource = resources.get(request.getWorkerId()); + NodeResources.update(resource, new NodeResource(request.getAvailableResource())); + } + done.run(response.setCommand(ResponseCommand.NORMAL).build()); + } else { + if (request.hasConnectionInfo()) { + membership.put(request.getWorkerId(), new NodeResource(request.getTotalResource())); + resources.put(request.getWorkerId(), new NodeResource(request.getAvailableResource())); + done.run(response.setCommand(ResponseCommand.NORMAL).build()); + } else { + done.run(response.setCommand(ResponseCommand.MEMBERSHIP).build()); + } + } + lastRequest = request; + barrier.countDown(); + } + } +} diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManagerService.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManagerService.java new file mode 100644 index 0000000000..49b1d2ba50 --- /dev/null +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManagerService.java @@ -0,0 +1,242 @@ +/** + * 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.tajo.worker; + +import com.google.common.collect.Lists; +import org.apache.hadoop.yarn.event.AsyncDispatcher; +import org.apache.tajo.*; +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.ipc.TajoWorkerProtocol; +import org.apache.tajo.master.cluster.WorkerConnectionInfo; +import org.apache.tajo.plan.serder.PlanProto; +import org.apache.tajo.resource.NodeResources; +import org.apache.tajo.rpc.CallFuture; +import org.apache.tajo.util.CommonTestingUtil; +import org.apache.tajo.worker.event.NodeResourceAllocateEvent; +import org.apache.tajo.worker.event.NodeResourceDeallocateEvent; +import org.junit.*; + +import java.util.Collections; +import java.util.List; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.junit.Assert.*; + +import static org.apache.tajo.ipc.TajoWorkerProtocol.*; +public class TestNodeResourceManagerService { + + private NodeResourceManagerService resourceManagerService; + private MockNodeStatusUpdaterService statusUpdater; + private AsyncDispatcher dispatcher; + private int taskMemory; + private TajoConf conf; + + @Before + public void setup() { + conf = new TajoConf(); + conf.set(CommonTestingUtil.TAJO_TEST_KEY, CommonTestingUtil.TAJO_TEST_TRUE); + + taskMemory = 512; + conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES, 4); + conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_MEMORY_MB, + taskMemory * conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES)); + conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_DISK_NUM, 4); + + dispatcher = new AsyncDispatcher(); + dispatcher.init(conf); + dispatcher.start(); + + resourceManagerService = new NodeResourceManagerService(dispatcher); + resourceManagerService.init(conf); + resourceManagerService.start(); + + WorkerConnectionInfo worker = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); + statusUpdater = new MockNodeStatusUpdaterService(new CountDownLatch(0), worker, resourceManagerService); + statusUpdater.init(conf); + statusUpdater.start(); + } + + @After + public void tearDown() { + resourceManagerService.stop(); + statusUpdater.stop(); + dispatcher.stop(); + } + + @Test + public void testNodeResourceAllocateEvent() throws Exception { + int requestSize = 4; + + CallFuture callFuture = new CallFuture(); + BatchAllocationRequestProto.Builder requestProto = BatchAllocationRequestProto.newBuilder(); + ExecutionBlockId ebId = new ExecutionBlockId(LocalTajoTestingUtility.newQueryId(), 0); + requestProto.setExecutionBlockId(ebId.getProto()); + + assertEquals(resourceManagerService.getTotalResource(), resourceManagerService.getAvailableResource()); + requestProto.addAllTaskRequest(createTaskRequests(taskMemory, requestSize)); + + dispatcher.getEventHandler().handle(new NodeResourceAllocateEvent(requestProto.build(), callFuture)); + + BatchAllocationResponseProto responseProto = callFuture.get(); + assertNotEquals(resourceManagerService.getTotalResource(), resourceManagerService.getAvailableResource()); + assertEquals(0, responseProto.getCancellationTaskCount()); + assertEquals(requestSize, resourceManagerService.getAllocatedSize()); + } + + + @Test + public void testNodeResourceCancellation() throws Exception { + int requestSize = conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES); + int overSize = 10; + + CallFuture callFuture = new CallFuture(); + BatchAllocationRequestProto.Builder requestProto = BatchAllocationRequestProto.newBuilder(); + ExecutionBlockId ebId = new ExecutionBlockId(LocalTajoTestingUtility.newQueryId(), 0); + requestProto.setExecutionBlockId(ebId.getProto()); + + assertEquals(resourceManagerService.getTotalResource(), resourceManagerService.getAvailableResource()); + requestProto.addAllTaskRequest(createTaskRequests(taskMemory, requestSize + overSize)); + + dispatcher.getEventHandler().handle(new NodeResourceAllocateEvent(requestProto.build(), callFuture)); + BatchAllocationResponseProto responseProto = callFuture.get(); + + assertEquals(overSize, responseProto.getCancellationTaskCount()); + assertEquals(requestSize, resourceManagerService.getAllocatedSize()); + } + + @Test + public void testNodeResourceDeallocateEvent() throws Exception { + int requestSize = 4; + + CallFuture callFuture = new CallFuture(); + BatchAllocationRequestProto.Builder requestProto = BatchAllocationRequestProto.newBuilder(); + ExecutionBlockId ebId = new ExecutionBlockId(LocalTajoTestingUtility.newQueryId(), 0); + requestProto.setExecutionBlockId(ebId.getProto()); + + assertEquals(resourceManagerService.getTotalResource(), resourceManagerService.getAvailableResource()); + requestProto.addAllTaskRequest(createTaskRequests(taskMemory, requestSize)); + + dispatcher.getEventHandler().handle(new NodeResourceAllocateEvent(requestProto.build(), callFuture)); + + BatchAllocationResponseProto responseProto = callFuture.get(); + assertNotEquals(resourceManagerService.getTotalResource(), resourceManagerService.getAvailableResource()); + assertEquals(0, responseProto.getCancellationTaskCount()); + assertEquals(requestSize, resourceManagerService.getAllocatedSize()); + + //deallocate + for(TaskAllocationRequestProto allocationRequestProto : requestProto.getTaskRequestList()) { + // direct invoke handler for testing + resourceManagerService.handle(new NodeResourceDeallocateEvent(allocationRequestProto.getResource())); + } + assertEquals(0, resourceManagerService.getAllocatedSize()); + assertEquals(resourceManagerService.getTotalResource(), resourceManagerService.getAvailableResource()); + } + + @Test(timeout = 30000) + public void testParallelRequest() throws Exception { + final int parallelCount = Runtime.getRuntime().availableProcessors(); + final int taskSize = 100000; + final AtomicInteger totalComplete = new AtomicInteger(); + final AtomicInteger totalCanceled = new AtomicInteger(); + + final ExecutionBlockId ebId = new ExecutionBlockId(LocalTajoTestingUtility.newQueryId(), 0); + final List totalTasks = + Collections.synchronizedList(createTaskRequests(taskMemory, taskSize)); + + ExecutorService executor = Executors.newFixedThreadPool(parallelCount); + List futureList = Lists.newArrayList(); + + long startTime = System.currentTimeMillis(); + for (int i = 0; i < parallelCount; i++) { + futureList.add(executor.submit(new Runnable() { + @Override + public void run() { + int complete = 0; + while (true) { + TaskAllocationRequestProto task; + synchronized (totalTasks) { + if (totalTasks.isEmpty()) break; + else { + task = totalTasks.remove(0); + } + } + + BatchAllocationRequestProto.Builder requestProto = BatchAllocationRequestProto.newBuilder(); + requestProto.addTaskRequest(task); + requestProto.setExecutionBlockId(ebId.getProto()); + + CallFuture callFuture = new CallFuture(); + dispatcher.getEventHandler().handle(new NodeResourceAllocateEvent(requestProto.build(), callFuture)); + try { + BatchAllocationResponseProto proto = callFuture.get(); + if (proto.getCancellationTaskCount() > 0) { + totalTasks.addAll(proto.getCancellationTaskList()); + totalCanceled.addAndGet(proto.getCancellationTaskCount()); + } else { + complete++; + dispatcher.getEventHandler().handle(new NodeResourceDeallocateEvent(task.getResource())); + } + } catch (Exception e) { + fail(e.getMessage()); + } + } + System.out.println(Thread.currentThread().getName() + " complete task: " + complete); + totalComplete.addAndGet(complete); + } + }) + ); + } + + for (Future future : futureList) { + future.get(); + } + + System.out.println(parallelCount + " Thread, completed requests: " + totalComplete.get() + ", canceled requests:" + + totalCanceled.get() + ", " + +(System.currentTimeMillis() - startTime) + " ms elapsed"); + executor.shutdown(); + assertEquals(taskSize, totalComplete.get()); + } + + protected static List createTaskRequests(int memory, int size) { + List requestProtoList = Lists.newArrayList(); + for (int i = 0; i < size; i++) { + + ExecutionBlockId nullStage = QueryIdFactory.newExecutionBlockId(QueryIdFactory.NULL_QUERY_ID, 0); + TaskAttemptId taskAttemptId = QueryIdFactory.newTaskAttemptId(QueryIdFactory.newTaskId(nullStage, i), 0); + + TajoWorkerProtocol.TaskRequestProto.Builder builder = + TajoWorkerProtocol.TaskRequestProto.newBuilder(); + builder.setId(taskAttemptId.getProto()); + builder.setShouldDie(true); + builder.setOutputTable(""); + builder.setPlan(PlanProto.LogicalNodeTree.newBuilder()); + builder.setClusteredOutput(false); + + + requestProtoList.add(TaskAllocationRequestProto.newBuilder() + .setResource(NodeResources.createResource(memory).getProto()) + .setTaskRequest(builder.build()).build()); + } + return requestProtoList; + } +} diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdaterService.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdaterService.java new file mode 100644 index 0000000000..ae0fdda2eb --- /dev/null +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdaterService.java @@ -0,0 +1,115 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.worker; + +import org.apache.hadoop.yarn.event.AsyncDispatcher; +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.ipc.TajoResourceTrackerProtocol; +import org.apache.tajo.master.cluster.WorkerConnectionInfo; +import org.apache.tajo.util.CommonTestingUtil; +import org.apache.tajo.worker.event.NodeStatusEvent; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.util.concurrent.CountDownLatch; +import static org.junit.Assert.*; + +public class TestNodeStatusUpdaterService { + + private NodeResourceManagerService resourceManagerService; + private MockNodeStatusUpdaterService statusUpdater; + private AsyncDispatcher dispatcher; + private TajoConf conf; + + @Before + public void setup() { + conf = new TajoConf(); + conf.set(CommonTestingUtil.TAJO_TEST_KEY, CommonTestingUtil.TAJO_TEST_TRUE); + + conf.setIntVar(TajoConf.ConfVars.WORKER_HEARTBEAT_INTERVAL, 1000); + dispatcher = new AsyncDispatcher(); + dispatcher.init(conf); + dispatcher.start(); + + resourceManagerService = new NodeResourceManagerService(dispatcher); + resourceManagerService.init(conf); + resourceManagerService.start(); + } + + @After + public void tearDown() { + resourceManagerService.stop(); + if (statusUpdater != null) statusUpdater.stop(); + dispatcher.stop(); + } + + @Test(timeout = 20000) + public void testNodeMembership() throws Exception { + CountDownLatch barrier = new CountDownLatch(1); + WorkerConnectionInfo worker = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); + statusUpdater = new MockNodeStatusUpdaterService(barrier, worker, resourceManagerService); + statusUpdater.init(conf); + statusUpdater.start(); + + MockNodeStatusUpdaterService.MockResourceTracker resourceTracker = statusUpdater.getResourceTracker(); + barrier.await(); + + assertTrue(resourceTracker.getTotalResource().containsKey(worker.getId())); + assertEquals(resourceManagerService.getTotalResource(), + resourceTracker.getTotalResource().get(worker.getId())); + + assertEquals(resourceManagerService.getAvailableResource(), + resourceTracker.getAvailableResource().get(worker.getId())); + } + + @Test(timeout = 20000) + public void testPing() throws Exception { + CountDownLatch barrier = new CountDownLatch(2); + WorkerConnectionInfo worker = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); + statusUpdater = new MockNodeStatusUpdaterService(barrier, worker, resourceManagerService); + statusUpdater.init(conf); + statusUpdater.start(); + + MockNodeStatusUpdaterService.MockResourceTracker resourceTracker = statusUpdater.getResourceTracker(); + barrier.await(); + + TajoResourceTrackerProtocol.NodeHeartbeatRequestProto lastRequest = resourceTracker.getLastRequest(); + assertTrue(lastRequest.hasWorkerId()); + assertFalse(lastRequest.hasAvailableResource()); + assertFalse(lastRequest.hasTotalResource()); + assertFalse(lastRequest.hasConnectionInfo()); + } + + @Test(timeout = 20000) + public void testResourceReport() throws Exception { + CountDownLatch barrier = new CountDownLatch(2); + WorkerConnectionInfo worker = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); + statusUpdater = new MockNodeStatusUpdaterService(barrier, worker, resourceManagerService); + statusUpdater.init(conf); + statusUpdater.start(); + + for (int i = 0; i < statusUpdater.getQueueingLimit(); i++) { + dispatcher.getEventHandler().handle(new NodeStatusEvent(NodeStatusEvent.EventType.REPORT_RESOURCE, + resourceManagerService.getAvailableResource())); + } + barrier.await(); + assertEquals(0, statusUpdater.getQueueSize()); + } +} From 24a845cdaf1467610c897e35ed6b98ece8cb7d02 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Tue, 19 May 2015 02:24:21 +0900 Subject: [PATCH 04/21] trigger CI --- CHANGES | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGES b/CHANGES index ba9daf2ca0..43e80c06e9 100644 --- a/CHANGES +++ b/CHANGES @@ -1,5 +1,6 @@ Tajo Change Log + Release 0.11.0 - unreleased NEW FEATURES From 798c1451efacba27ac50e25baf9fef95af9a4880 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Tue, 19 May 2015 10:51:43 +0900 Subject: [PATCH 05/21] TAJO-1599 --- .../src/main/java/org/apache/tajo/conf/TajoConf.java | 4 +++- .../org/apache/tajo/worker/NodeResourceManagerService.java | 6 ++++-- .../apache/tajo/worker/TestNodeResourceManagerService.java | 3 ++- 3 files changed, 9 insertions(+), 4 deletions(-) diff --git a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java index 995b8d639c..e20658b1a9 100644 --- a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java +++ b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java @@ -176,7 +176,9 @@ public static enum ConfVars implements ConfigKey { WORKER_RESOURCE_AVAILABLE_MEMORY_MB("tajo.worker.resource.memory-mb", 1024, Validators.min("64")), @Deprecated WORKER_RESOURCE_AVAILABLE_DISKS("tajo.worker.resource.disks", 1.0f), - WORKER_RESOURCE_AVAILABLE_DISK_NUM("tajo.worker.resource.disk.num", 1), + WORKER_RESOURCE_AVAILABLE_DISKS_NUM("tajo.worker.resource.disks.num", 1, Validators.min("1")), + WORKER_RESOURCE_AVAILABLE_DISK_PARALLEL_NUM("tajo.worker.resource.disk.parallel-execution.num", 2, + Validators.min("1")), WORKER_EXECUTION_MAX_SLOTS("tajo.worker.parallel-execution.max-num", 2), WORKER_RESOURCE_DFS_DIR_AWARE("tajo.worker.resource.dfs-dir-aware", false, Validators.bool()), diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManagerService.java b/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManagerService.java index c4c57cb628..26bdba5006 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManagerService.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManagerService.java @@ -138,12 +138,14 @@ private NodeResource createWorkerResource(TajoConf conf) { } int vCores = conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES); - int disks = conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_DISK_NUM); + int disks = conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_DISKS_NUM); int dataNodeStorageSize = DiskUtil.getDataNodeStorageSize(); if (conf.getBoolVar(TajoConf.ConfVars.WORKER_RESOURCE_DFS_DIR_AWARE) && dataNodeStorageSize > 0) { disks = dataNodeStorageSize; } - return NodeResource.createResource(memoryMb, disks, vCores); + + int diskParallels = conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_DISK_PARALLEL_NUM); + return NodeResource.createResource(memoryMb, disks * diskParallels, vCores); } } diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManagerService.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManagerService.java index 49b1d2ba50..0bcaedb266 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManagerService.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManagerService.java @@ -60,7 +60,8 @@ public void setup() { conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES, 4); conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_MEMORY_MB, taskMemory * conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES)); - conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_DISK_NUM, 4); + conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_DISKS_NUM, 4); + conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_DISK_PARALLEL_NUM, 1); dispatcher = new AsyncDispatcher(); dispatcher.init(conf); From 9b7a9f30e48e51d79ed85264a762d71836e10c66 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Tue, 19 May 2015 10:56:06 +0900 Subject: [PATCH 06/21] I can't rename file. maybe git bug --- .../apache/tajo/worker/event/NodeResourceDeAllocateEvent.java | 1 + 1 file changed, 1 insertion(+) diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceDeAllocateEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceDeAllocateEvent.java index 78a7796b33..a298d77e20 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceDeAllocateEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceDeAllocateEvent.java @@ -28,6 +28,7 @@ public class NodeResourceDeallocateEvent extends NodeResourceManagerEvent { public NodeResourceDeallocateEvent(TajoProtos.NodeResourceProto proto) { this(new NodeResource(proto)); } + public NodeResourceDeallocateEvent(NodeResource resource) { super(EventType.DEALLOCATE); this.resource = resource; From 9b40d944e5252cd3c34350ddb2866f8a6cf60b21 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Tue, 19 May 2015 11:06:30 +0900 Subject: [PATCH 07/21] delete old file --- .../event/NodeResourceDeAllocateEvent.java | 40 ------------------- 1 file changed, 40 deletions(-) delete mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceDeAllocateEvent.java diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceDeAllocateEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceDeAllocateEvent.java deleted file mode 100644 index a298d77e20..0000000000 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceDeAllocateEvent.java +++ /dev/null @@ -1,40 +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.tajo.worker.event; - -import org.apache.tajo.TajoProtos; -import org.apache.tajo.resource.NodeResource; - -public class NodeResourceDeallocateEvent extends NodeResourceManagerEvent { - - private NodeResource resource; - - public NodeResourceDeallocateEvent(TajoProtos.NodeResourceProto proto) { - this(new NodeResource(proto)); - } - - public NodeResourceDeallocateEvent(NodeResource resource) { - super(EventType.DEALLOCATE); - this.resource = resource; - } - - public NodeResource getResource() { - return resource; - } -} From 4448a0bd8ecde82225ea49b72cb015d781e1a0ad Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Tue, 19 May 2015 11:07:17 +0900 Subject: [PATCH 08/21] add renamed file --- .../event/NodeResourceDeallocateEvent.java | 40 +++++++++++++++++++ 1 file changed, 40 insertions(+) create mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceDeallocateEvent.java diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceDeallocateEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceDeallocateEvent.java new file mode 100644 index 0000000000..a298d77e20 --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceDeallocateEvent.java @@ -0,0 +1,40 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.worker.event; + +import org.apache.tajo.TajoProtos; +import org.apache.tajo.resource.NodeResource; + +public class NodeResourceDeallocateEvent extends NodeResourceManagerEvent { + + private NodeResource resource; + + public NodeResourceDeallocateEvent(TajoProtos.NodeResourceProto proto) { + this(new NodeResource(proto)); + } + + public NodeResourceDeallocateEvent(NodeResource resource) { + super(EventType.DEALLOCATE); + this.resource = resource; + } + + public NodeResource getResource() { + return resource; + } +} From 5d75eb3d20ec9ba29d21e7f626a08dbd689e7505 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Tue, 19 May 2015 11:09:28 +0900 Subject: [PATCH 09/21] revert changes --- CHANGES | 1 - 1 file changed, 1 deletion(-) diff --git a/CHANGES b/CHANGES index 43e80c06e9..ba9daf2ca0 100644 --- a/CHANGES +++ b/CHANGES @@ -1,6 +1,5 @@ Tajo Change Log - Release 0.11.0 - unreleased NEW FEATURES From ec57a13f0349f29135a00b02fb24c0a70006a40f Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Tue, 19 May 2015 23:47:15 +0900 Subject: [PATCH 10/21] change to atomic updater and update test case --- .../apache/tajo/resource/NodeResource.java | 40 ++++++++++++++----- .../worker/NodeResourceManagerService.java | 4 +- .../TestNodeResourceManagerService.java | 26 +++++------- 3 files changed, 42 insertions(+), 28 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/resource/NodeResource.java b/tajo-core/src/main/java/org/apache/tajo/resource/NodeResource.java index c2512b971c..f51fc07e12 100644 --- a/tajo-core/src/main/java/org/apache/tajo/resource/NodeResource.java +++ b/tajo-core/src/main/java/org/apache/tajo/resource/NodeResource.java @@ -19,9 +19,12 @@ package org.apache.tajo.resource; import com.google.common.base.Objects; +import io.netty.util.internal.PlatformDependent; import org.apache.tajo.TajoProtos; import org.apache.tajo.common.ProtoObject; +import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; + /** *

NodeResource models a set of computer resources in the * cluster.

@@ -37,14 +40,30 @@ public class NodeResource implements ProtoObject, Comparable { - private int memory; - private int disks; - private int vCores; + private volatile int memory; + private volatile int disks; + private volatile int vCores; + + private static AtomicIntegerFieldUpdater MEMORY_UPDATER; + private static AtomicIntegerFieldUpdater DISKS_UPDATER; + private static AtomicIntegerFieldUpdater VCORES_UPDATER; + + static { + MEMORY_UPDATER = PlatformDependent.newAtomicIntegerFieldUpdater(NodeResource.class, "memory"); + if (MEMORY_UPDATER == null) { + MEMORY_UPDATER = AtomicIntegerFieldUpdater.newUpdater(NodeResource.class, "memory"); + DISKS_UPDATER = AtomicIntegerFieldUpdater.newUpdater(NodeResource.class, "disks"); + VCORES_UPDATER = AtomicIntegerFieldUpdater.newUpdater(NodeResource.class, "vCores"); + } else { + DISKS_UPDATER = PlatformDependent.newAtomicIntegerFieldUpdater(NodeResource.class, "disks"); + VCORES_UPDATER = PlatformDependent.newAtomicIntegerFieldUpdater(NodeResource.class, "vCores"); + } + } public NodeResource(TajoProtos.NodeResourceProto proto) { - this.memory = proto.getMemory(); - this.vCores = proto.getVirtualCores(); - this.disks = proto.getDisks(); + setMemory(proto.getMemory()); + setDisks(proto.getDisks()); + setVirtualCores(proto.getVirtualCores()); } private NodeResource() { @@ -69,8 +88,9 @@ public int getMemory() { * * @param memory memory of the resource */ + @SuppressWarnings("unchecked") public NodeResource setMemory(int memory) { - this.memory = memory; + MEMORY_UPDATER.lazySet(this, memory); return this; } @@ -89,8 +109,9 @@ public int getDisks() { * * @param disks number of disks of the resource */ + @SuppressWarnings("unchecked") public NodeResource setDisks(int disks) { - this.disks = disks; + DISKS_UPDATER.lazySet(this, disks); return this; } @@ -111,8 +132,9 @@ public int getVirtualCores() { * * @param vCores number of virtual cpu cores of the resource */ + @SuppressWarnings("unchecked") public NodeResource setVirtualCores(int vCores) { - this.vCores = vCores; + VCORES_UPDATER.lazySet(this, vCores); return this; } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManagerService.java b/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManagerService.java index 26bdba5006..4b070223cb 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManagerService.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManagerService.java @@ -103,11 +103,11 @@ protected Dispatcher getDispatcher() { } protected NodeResource getTotalResource() { - return NodeResources.clone(totalResource); + return totalResource; } protected NodeResource getAvailableResource() { - return NodeResources.clone(availableResource); + return availableResource; } public int getAllocatedSize() { diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManagerService.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManagerService.java index 0bcaedb266..ea6c2390b6 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManagerService.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManagerService.java @@ -32,12 +32,9 @@ import org.apache.tajo.worker.event.NodeResourceDeallocateEvent; import org.junit.*; -import java.util.Collections; import java.util.List; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; +import java.util.Queue; +import java.util.concurrent.*; import java.util.concurrent.atomic.AtomicInteger; import static org.junit.Assert.*; @@ -161,8 +158,7 @@ public void testParallelRequest() throws Exception { final AtomicInteger totalCanceled = new AtomicInteger(); final ExecutionBlockId ebId = new ExecutionBlockId(LocalTajoTestingUtility.newQueryId(), 0); - final List totalTasks = - Collections.synchronizedList(createTaskRequests(taskMemory, taskSize)); + final Queue totalTasks = createTaskRequests(taskMemory, taskSize); ExecutorService executor = Executors.newFixedThreadPool(parallelCount); List futureList = Lists.newArrayList(); @@ -174,13 +170,9 @@ public void testParallelRequest() throws Exception { public void run() { int complete = 0; while (true) { - TaskAllocationRequestProto task; - synchronized (totalTasks) { - if (totalTasks.isEmpty()) break; - else { - task = totalTasks.remove(0); - } - } + TaskAllocationRequestProto task = totalTasks.poll(); + if (task == null) break; + BatchAllocationRequestProto.Builder requestProto = BatchAllocationRequestProto.newBuilder(); requestProto.addTaskRequest(task); @@ -201,7 +193,7 @@ public void run() { fail(e.getMessage()); } } - System.out.println(Thread.currentThread().getName() + " complete task: " + complete); + System.out.println(Thread.currentThread().getName() + " complete requests: " + complete); totalComplete.addAndGet(complete); } }) @@ -218,8 +210,8 @@ public void run() { assertEquals(taskSize, totalComplete.get()); } - protected static List createTaskRequests(int memory, int size) { - List requestProtoList = Lists.newArrayList(); + protected static Queue createTaskRequests(int memory, int size) { + Queue requestProtoList = new LinkedBlockingQueue(); for (int i = 0; i < size; i++) { ExecutionBlockId nullStage = QueryIdFactory.newExecutionBlockId(QueryIdFactory.NULL_QUERY_ID, 0); From 9902e57fc0b613b8a7ebad22b801ced8bdf680f4 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Wed, 20 May 2015 00:04:33 +0900 Subject: [PATCH 11/21] remove findbugs warnings --- .../worker/NodeResourceManagerService.java | 45 +++++++++---------- .../TestNodeResourceManagerService.java | 2 +- 2 files changed, 22 insertions(+), 25 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManagerService.java b/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManagerService.java index 4b070223cb..08d6b5abc6 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManagerService.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManagerService.java @@ -69,32 +69,29 @@ protected void serviceInit(Configuration conf) throws Exception { @Override public void handle(NodeResourceManagerEvent event) { - switch (event.getType()) { - case ALLOCATE: { - NodeResourceAllocateEvent allocateEvent = (NodeResourceAllocateEvent) event; - BatchAllocationResponseProto.Builder response = BatchAllocationResponseProto.newBuilder(); - for (TaskAllocationRequestProto request : allocateEvent.getRequest().getTaskRequestList()) { - NodeResource resource = new NodeResource(request.getResource()); - if (allocate(resource)) { - allocatedSize.incrementAndGet(); - //TODO send task event to taskExecutorService - } else { - response.addCancellationTask(request); - } + + if (event instanceof NodeResourceAllocateEvent) { + NodeResourceAllocateEvent allocateEvent = (NodeResourceAllocateEvent) event; + BatchAllocationResponseProto.Builder response = BatchAllocationResponseProto.newBuilder(); + for (TaskAllocationRequestProto request : allocateEvent.getRequest().getTaskRequestList()) { + NodeResource resource = new NodeResource(request.getResource()); + if (allocate(resource)) { + allocatedSize.incrementAndGet(); + //TODO send task event to taskExecutorService + } else { + response.addCancellationTask(request); } - allocateEvent.getCallback().run(response.build()); - break; - } - case DEALLOCATE: { - allocatedSize.decrementAndGet(); - NodeResourceDeallocateEvent deallocateEvent = (NodeResourceDeallocateEvent) event; - release(deallocateEvent.getResource()); - - // send current resource to ResourceTracker - getDispatcher().getEventHandler().handle( - new NodeStatusEvent(NodeStatusEvent.EventType.REPORT_RESOURCE, getAvailableResource())); - break; } + allocateEvent.getCallback().run(response.build()); + + } else if (event instanceof NodeResourceDeallocateEvent) { + allocatedSize.decrementAndGet(); + NodeResourceDeallocateEvent deallocateEvent = (NodeResourceDeallocateEvent) event; + release(deallocateEvent.getResource()); + + // send current resource to ResourceTracker + getDispatcher().getEventHandler().handle( + new NodeStatusEvent(NodeStatusEvent.EventType.REPORT_RESOURCE, getAvailableResource())); } } diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManagerService.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManagerService.java index ea6c2390b6..7586f37ec2 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManagerService.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManagerService.java @@ -152,7 +152,7 @@ public void testNodeResourceDeallocateEvent() throws Exception { @Test(timeout = 30000) public void testParallelRequest() throws Exception { - final int parallelCount = Runtime.getRuntime().availableProcessors(); + final int parallelCount = conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES) * 2; final int taskSize = 100000; final AtomicInteger totalComplete = new AtomicInteger(); final AtomicInteger totalCanceled = new AtomicInteger(); From 3543eaabba6aa2b57011e4484e9a25272d938f86 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Wed, 20 May 2015 00:05:46 +0900 Subject: [PATCH 12/21] TAJO-1615 --- .../java/org/apache/tajo/conf/TajoConf.java | 3 +- .../tajo/worker/ExecutionBlockContext.java | 11 ++ .../worker/NodeResourceManagerService.java | 9 +- .../org/apache/tajo/worker/TajoWorker.java | 1 + .../org/apache/tajo/worker/TaskContainer.java | 97 +++++++++++++++ .../tajo/worker/TaskExecutorService.java | 114 ++++++++++++++++++ .../apache/tajo/worker/event/TaskEvent.java | 42 +++++++ .../tajo/worker/event/TaskStartEvent.java | 44 +++++++ 8 files changed, 315 insertions(+), 6 deletions(-) create mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/TaskContainer.java create mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutorService.java create mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/event/TaskEvent.java create mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/event/TaskStartEvent.java diff --git a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java index e20658b1a9..0382031382 100644 --- a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java +++ b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java @@ -216,7 +216,8 @@ public static enum ConfVars implements ConfigKey { PULLSERVER_PORT("tajo.pullserver.port", 0, Validators.range("0", "65535")), SHUFFLE_SSL_ENABLED_KEY("tajo.pullserver.ssl.enabled", false, Validators.bool()), SHUFFLE_FILE_FORMAT("tajo.shuffle.file-format", "RAW"), - SHUFFLE_FETCHER_PARALLEL_EXECUTION_MAX_NUM("tajo.shuffle.fetcher.parallel-execution.max-num", 2), + SHUFFLE_FETCHER_PARALLEL_EXECUTION_MAX_NUM("tajo.shuffle.fetcher.parallel-execution.max-num", + Runtime.getRuntime().availableProcessors() * 2, Validators.min("1")), SHUFFLE_FETCHER_CHUNK_MAX_SIZE("tajo.shuffle.fetcher.chunk.max-size", 8192), SHUFFLE_FETCHER_READ_TIMEOUT("tajo.shuffle.fetcher.read.timeout-sec", 120), SHUFFLE_FETCHER_READ_RETRY_MAX_NUM("tajo.shuffle.fetcher.read.retry.max-num", 20), diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java b/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java index 0cc330468f..e97955311e 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java @@ -265,6 +265,17 @@ public void addTaskHistory(String taskRunnerId, TaskAttemptId quAttemptId, TaskH histories.get(taskRunnerId).addTaskHistory(quAttemptId, taskHistory); } + public void fatalError(TaskAttemptId taskAttemptId, String message) { + if (message == null) { + message = "No error message"; + } + TaskFatalErrorReport.Builder builder = TaskFatalErrorReport.newBuilder() + .setId(taskAttemptId.getProto()) + .setErrorMessage(message); + + getStub().fatalError(null, builder.build(), NullCallback.get()); + } + public TaskRunnerHistory createTaskRunnerHistory(TaskRunner runner){ histories.putIfAbsent(runner.getId(), new TaskRunnerHistory(runner.getContainerId(), executionBlockId)); return histories.get(runner.getId()); diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManagerService.java b/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManagerService.java index 26bdba5006..45b22669d8 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManagerService.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManagerService.java @@ -30,10 +30,7 @@ import org.apache.tajo.storage.DiskUtil; import org.apache.tajo.unit.StorageUnit; import org.apache.tajo.util.CommonTestingUtil; -import org.apache.tajo.worker.event.NodeResourceAllocateEvent; -import org.apache.tajo.worker.event.NodeResourceDeallocateEvent; -import org.apache.tajo.worker.event.NodeResourceManagerEvent; -import org.apache.tajo.worker.event.NodeStatusEvent; +import org.apache.tajo.worker.event.*; import java.util.concurrent.atomic.AtomicInteger; @@ -77,8 +74,10 @@ public void handle(NodeResourceManagerEvent event) { NodeResource resource = new NodeResource(request.getResource()); if (allocate(resource)) { allocatedSize.incrementAndGet(); - //TODO send task event to taskExecutorService + //send task start event to TaskExecutorService + getDispatcher().getEventHandler().handle(new TaskStartEvent(request.getTaskRequest(), resource)); } else { + // reject the exceeded requests response.addCancellationTask(request); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java index 4f07ca67d7..6f4e89919a 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java @@ -627,6 +627,7 @@ public static List getMountPath() throws IOException { } public static void main(String[] args) throws Exception { + Thread.setDefaultUncaughtExceptionHandler(new TajoUncaughtExceptionHandler()); StringUtils.startupShutdownMessage(TajoWorker.class, args, LOG); TajoConf tajoConf = new TajoConf(); diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskContainer.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskContainer.java new file mode 100644 index 0000000000..e6fd62dd2e --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskContainer.java @@ -0,0 +1,97 @@ +/** + * 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.tajo.worker; + +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.service.AbstractService; +import org.apache.tajo.ExecutionBlockId; +import org.apache.tajo.TaskAttemptId; +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.conf.TajoConf.ConfVars; +import org.apache.tajo.engine.query.TaskRequestImpl; +import org.apache.tajo.ipc.QueryMasterProtocol.QueryMasterProtocolService; +import org.apache.tajo.master.container.TajoContainerId; +import org.apache.tajo.master.container.TajoContainerIdPBImpl; +import org.apache.tajo.master.container.TajoConverterUtils; +import org.apache.tajo.rpc.CallFuture; +import org.apache.tajo.rpc.NullCallback; + +import java.io.IOException; +import java.util.concurrent.*; + +import static org.apache.tajo.ipc.TajoWorkerProtocol.*; + +/** + * The driver class for Tajo Task processing. + */ +public class TaskContainer implements Runnable { + private static final Log LOG = LogFactory.getLog(TaskContainer.class); + + // Contains the object references related for TaskRunner + private ExecutionBlockContext executionBlockContext; + private Task task = null; + + public TaskContainer(ExecutionBlockContext executionBlockContext, Task task) { + this.executionBlockContext = executionBlockContext; + this.task = task; + } + + public void init() throws IOException { + if (executionBlockContext.isStopped()) return; + + LOG.info("Initializing: " + task.getId()); + getContext().getWorkerContext().getWorkerSystemMetrics().counter("query", "task").inc(); + task.init(); + } + + public void stop() { + task.cleanupTask(); + } + + public Task getTask() { + return task; + } + + public ExecutionBlockContext getContext() { + return executionBlockContext; + } + + @Override + public void run() { + if (executionBlockContext.isStopped()) return; + + try { + if (task.hasFetchPhase()) { + task.fetch(); // The fetch is performed in an asynchronous way. + } + if (!executionBlockContext.isStopped()) { + task.run(); + } + } catch (Exception t) { + LOG.error(t.getMessage(), t); + getContext().fatalError(task.getId(), t.getMessage()); + } finally { + task.cleanupTask(); + } + } +} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutorService.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutorService.java new file mode 100644 index 0000000000..b8dc55c9cb --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutorService.java @@ -0,0 +1,114 @@ +/** + * 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.tajo.worker; + +import com.google.common.collect.Maps; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.tajo.ExecutionBlockId; +import org.apache.tajo.TaskId; +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.conf.TajoConf.ConfVars; +import org.apache.tajo.resource.NodeResource; +import org.apache.tajo.worker.event.TaskEvent; +import org.apache.tajo.worker.event.TaskStartEvent; + +import java.io.IOException; +import java.util.concurrent.*; + +import static org.apache.tajo.ipc.TajoWorkerProtocol.*; + +public class TaskExecutorService extends ThreadPoolExecutor implements EventHandler { + private static final Log LOG = LogFactory.getLog(TaskExecutorService.class); + + private final TajoConf conf; + private final NodeResourceManagerService resourceManagerService; + + private final ConcurrentMap executionBlockContextMap; + private final ConcurrentMap allocatedResourceMap; + private final ThreadPoolExecutor fetcherExecutor; + + public TaskExecutorService(TajoConf conf, NodeResourceManagerService resourceManagerService) { + this(conf, conf.getIntVar(ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES), resourceManagerService); + + } + + public TaskExecutorService(TajoConf conf, int nThreads, NodeResourceManagerService resourceManagerService) { + super(nThreads, nThreads, + 0L, TimeUnit.MILLISECONDS, + new LinkedBlockingQueue(), + new ThreadFactoryBuilder().setNameFormat("Task executor #%d").build()); + this.conf = conf; + this.resourceManagerService = resourceManagerService; + this.executionBlockContextMap = Maps.newConcurrentMap(); + this.allocatedResourceMap = Maps.newConcurrentMap(); + + int maxFetcherThreads = conf.getIntVar(ConfVars.SHUFFLE_FETCHER_PARALLEL_EXECUTION_MAX_NUM); + this.fetcherExecutor = new ThreadPoolExecutor(Math.min(nThreads, maxFetcherThreads), + maxFetcherThreads, + 60L, TimeUnit.SECONDS, + new SynchronousQueue(true)); + LOG.info("Startup TaskExecutorService"); + } + + @Override + protected void beforeExecute(Thread t, Runnable r) { + TaskContainer container = (T) r; + try { + container.init(); + } catch (IOException e) { + LOG.error(e.getMessage(), e); + container.getContext().fatalError(container.getTask().getId(), e.getMessage()); + } + } + + @Override + protected void afterExecute(Runnable r, Throwable t) { + TaskContainer container = (T) r; + + if (container != null) { + LOG.error(t.getMessage(), t); + container.getContext().fatalError(container.getTask().getId(), t.getMessage()); + return; + } + + try { + container.stop(); + } catch (Throwable throwable) { + LOG.error(t.getMessage(), t); + container.getContext().fatalError(container.getTask().getId(), t.getMessage()); + } + } + + protected ExecutorService getFetcherExecutor() { + return fetcherExecutor; + } + + @Override + public void handle(TaskEvent event) { + switch (event.getType()) { + case START:{ + TaskStartEvent startEvent = (TaskStartEvent)event; + allocatedResourceMap.putIfAbsent() + } + } + } +} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskEvent.java new file mode 100644 index 0000000000..3a245f6aee --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskEvent.java @@ -0,0 +1,42 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.worker.event; + +import org.apache.hadoop.yarn.event.AbstractEvent; +import org.apache.tajo.TaskAttemptId; + +public class TaskEvent extends AbstractEvent { + public enum EventType { + START, + KILL, + ABORT + } + + private TaskAttemptId taskAttemptId; + + public TaskEvent(EventType eventType, + TaskAttemptId taskAttemptId) { + super(eventType); + this.taskAttemptId = taskAttemptId; + } + + public TaskAttemptId getTaskId() { + return taskAttemptId; + } +} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskStartEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskStartEvent.java new file mode 100644 index 0000000000..049ef665f8 --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskStartEvent.java @@ -0,0 +1,44 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.worker.event; + +import org.apache.tajo.TaskAttemptId; +import org.apache.tajo.resource.NodeResource; +import static org.apache.tajo.ipc.TajoWorkerProtocol.TaskRequestProto; + +public class TaskStartEvent extends TaskEvent { + + private NodeResource allocatedResource; + private TaskRequestProto taskRequest; + + public TaskStartEvent(TaskRequestProto taskRequest, + NodeResource allocatedResource) { + super(EventType.START, new TaskAttemptId(taskRequest.getId())); + this.taskRequest = taskRequest; + this.allocatedResource = allocatedResource; + } + + public NodeResource getAllocatedResource() { + return allocatedResource; + } + + public TaskRequestProto getTaskRequest() { + return taskRequest; + } +} From fb67da021a9d751cb2e9c4aa4da61d0d3b1cf512 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Wed, 20 May 2015 00:06:24 +0900 Subject: [PATCH 13/21] TAJO-1615 --- .../util/TajoUncaughtExceptionHandler.java | 70 +++++++++++ .../tajo/worker/TaskManagerService.java | 112 ++++++++++++++++++ 2 files changed, 182 insertions(+) create mode 100644 tajo-core/src/main/java/org/apache/tajo/util/TajoUncaughtExceptionHandler.java create mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/TaskManagerService.java diff --git a/tajo-core/src/main/java/org/apache/tajo/util/TajoUncaughtExceptionHandler.java b/tajo-core/src/main/java/org/apache/tajo/util/TajoUncaughtExceptionHandler.java new file mode 100644 index 0000000000..c424154ae8 --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/util/TajoUncaughtExceptionHandler.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.tajo.util; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.util.ExitUtil; +import org.apache.hadoop.util.ShutdownHookManager; + +import java.lang.Thread.UncaughtExceptionHandler; + +/** + * This class is intended to be installed by calling + * {@link Thread#setDefaultUncaughtExceptionHandler(UncaughtExceptionHandler)} + * In the main entry point. It is intended to try and cleanly shut down + * programs using the Yarn Event framework. + * + * Note: Right now it only will shut down the program if a Error is caught, but + * not any other exception. Anything else is just logged. + * + * this is an implementation copied from YarnUncaughtExceptionHandler + */ +public class TajoUncaughtExceptionHandler implements UncaughtExceptionHandler { + private static final Log LOG = LogFactory.getLog(TajoUncaughtExceptionHandler.class); + + @Override + public void uncaughtException(Thread t, Throwable e) { + if(ShutdownHookManager.get().isShutdownInProgress()) { + LOG.error("Thread " + t + " threw an Throwable, but we are shutting " + + "down, so ignoring this", e); + } else if(e instanceof Error) { + try { + LOG.fatal("Thread " + t + " threw an Error.", e); + } catch (Throwable err) { + //We don't want to not exit because of an issue with logging + } + + if(e instanceof OutOfMemoryError) { + //After catching an OOM java says it is undefined behavior, so don't + //even try to clean up or we can get stuck on shutdown. + try { + System.err.println("Halting due to Out Of Memory Error..."); + } catch (Throwable err) { + //Again we done want to exit because of logging issues. + } + ExitUtil.halt(-1); + } else { + //ExitUtil.terminate(-1); + } + } else { + LOG.error("Thread " + t + " threw an Exception.", e); + } + } +} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskManagerService.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskManagerService.java new file mode 100644 index 0000000000..b04834ea6c --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskManagerService.java @@ -0,0 +1,112 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.worker; + +import com.google.common.collect.Maps; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.tajo.ExecutionBlockId; +import org.apache.tajo.TaskId; +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.conf.TajoConf.ConfVars; +import org.apache.tajo.resource.NodeResource; +import org.apache.tajo.worker.event.TaskEvent; +import org.apache.tajo.worker.event.TaskStartEvent; + +import java.io.IOException; +import java.util.concurrent.*; + +public class TaskManagerService implements EventHandler { + private static final Log LOG = LogFactory.getLog(TaskManagerService.class); + + private final TajoConf conf; + private final NodeResourceManagerService resourceManagerService; + + private final ConcurrentMap executionBlockContextMap; + private final ConcurrentMap allocatedResourceMap; + private final ThreadPoolExecutor fetcherExecutor; + + public TaskManagerService(TajoConf conf, NodeResourceManagerService resourceManagerService) { + this(conf, conf.getIntVar(ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES), resourceManagerService); + + } + + public TaskManagerService(TajoConf conf, int nThreads, NodeResourceManagerService resourceManagerService) { + super(nThreads, nThreads, + 0L, TimeUnit.MILLISECONDS, + new LinkedBlockingQueue(), + new ThreadFactoryBuilder().setNameFormat("Task executor #%d").build()); + this.conf = conf; + this.resourceManagerService = resourceManagerService; + this.executionBlockContextMap = Maps.newConcurrentMap(); + this.allocatedResourceMap = Maps.newConcurrentMap(); + + int maxFetcherThreads = conf.getIntVar(ConfVars.SHUFFLE_FETCHER_PARALLEL_EXECUTION_MAX_NUM); + this.fetcherExecutor = new ThreadPoolExecutor(Math.min(nThreads, maxFetcherThreads), + maxFetcherThreads, + 60L, TimeUnit.SECONDS, + new SynchronousQueue(true)); + LOG.info("Startup TaskExecutorService"); + } + + @Override + protected void beforeExecute(Thread t, Runnable r) { + TaskContainer container = (T) r; + try { + container.init(); + } catch (IOException e) { + LOG.error(e.getMessage(), e); + container.getContext().fatalError(container.getTask().getId(), e.getMessage()); + } + } + + @Override + protected void afterExecute(Runnable r, Throwable t) { + TaskContainer container = (T) r; + + if (container != null) { + LOG.error(t.getMessage(), t); + container.getContext().fatalError(container.getTask().getId(), t.getMessage()); + return; + } + + try { + container.stop(); + } catch (Throwable throwable) { + LOG.error(t.getMessage(), t); + container.getContext().fatalError(container.getTask().getId(), t.getMessage()); + } + } + + protected ExecutorService getFetcherExecutor() { + return fetcherExecutor; + } + + @Override + public void handle(TaskEvent event) { + switch (event.getType()) { + case START:{ + TaskStartEvent startEvent = (TaskStartEvent)event; + allocatedResourceMap.putIfAbsent() + } + } + } +} From 1d33d36c917f1f1bb376225dcccf18f61568fcc9 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Wed, 20 May 2015 13:59:53 +0900 Subject: [PATCH 14/21] rename service to manager --- ...rService.java => NodeResourceManager.java} | 12 +++--- ...terService.java => NodeStatusUpdater.java} | 20 +++++----- ...ervice.java => MockNodeStatusUpdater.java} | 8 ++-- ...vice.java => TestNodeResourceManager.java} | 38 +++++++++---------- ...ervice.java => TestNodeStatusUpdater.java} | 30 +++++++-------- 5 files changed, 54 insertions(+), 54 deletions(-) rename tajo-core/src/main/java/org/apache/tajo/worker/{NodeResourceManagerService.java => NodeResourceManager.java} (93%) rename tajo-core/src/main/java/org/apache/tajo/worker/{NodeStatusUpdaterService.java => NodeStatusUpdater.java} (92%) rename tajo-core/src/test/java/org/apache/tajo/worker/{MockNodeStatusUpdaterService.java => MockNodeStatusUpdater.java} (92%) rename tajo-core/src/test/java/org/apache/tajo/worker/{TestNodeResourceManagerService.java => TestNodeResourceManager.java} (85%) rename tajo-core/src/test/java/org/apache/tajo/worker/{TestNodeStatusUpdaterService.java => TestNodeStatusUpdater.java} (77%) diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManagerService.java b/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java similarity index 93% rename from tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManagerService.java rename to tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java index 08d6b5abc6..20eec6b4e2 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManagerService.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java @@ -39,8 +39,8 @@ import static org.apache.tajo.ipc.TajoWorkerProtocol.*; -public class NodeResourceManagerService extends AbstractService implements EventHandler { - private static final Log LOG = LogFactory.getLog(NodeResourceManagerService.class); +public class NodeResourceManager extends AbstractService implements EventHandler { + private static final Log LOG = LogFactory.getLog(NodeResourceManager.class); private final Dispatcher dispatcher; private NodeResource totalResource; @@ -48,8 +48,8 @@ public class NodeResourceManagerService extends AbstractService implements Event private AtomicInteger allocatedSize; private TajoConf tajoConf; - public NodeResourceManagerService(Dispatcher dispatcher){ - super(NodeResourceManagerService.class.getName()); + public NodeResourceManager(Dispatcher dispatcher){ + super(NodeResourceManager.class.getName()); this.dispatcher = dispatcher; } @@ -64,7 +64,7 @@ protected void serviceInit(Configuration conf) throws Exception { this.dispatcher.register(NodeResourceManagerEvent.EventType.class, this); this.allocatedSize = new AtomicInteger(); super.serviceInit(conf); - LOG.info("Initialized NodeResourceManagerService for " + totalResource); + LOG.info("Initialized NodeResourceManager for " + totalResource); } @Override @@ -77,7 +77,7 @@ public void handle(NodeResourceManagerEvent event) { NodeResource resource = new NodeResource(request.getResource()); if (allocate(resource)) { allocatedSize.incrementAndGet(); - //TODO send task event to taskExecutorService + //TODO send task event to taskExecutor } else { response.addCancellationTask(request); } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdaterService.java b/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java similarity index 92% rename from tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdaterService.java rename to tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java index b16b46174c..84ac419066 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdaterService.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java @@ -48,9 +48,9 @@ /** * It periodically sends heartbeat to {@link org.apache.tajo.master.rm.TajoResourceTracker} via asynchronous rpc. */ -public class NodeStatusUpdaterService extends AbstractService implements EventHandler { +public class NodeStatusUpdater extends AbstractService implements EventHandler { - private final static Log LOG = LogFactory.getLog(NodeStatusUpdaterService.class); + private final static Log LOG = LogFactory.getLog(NodeStatusUpdater.class); private TajoConf tajoConf; private StatusUpdaterThread updaterThread; @@ -58,16 +58,16 @@ public class NodeStatusUpdaterService extends AbstractService implements EventHa private volatile long heartBeatInterval; private BlockingQueue heartBeatRequestQueue; private final WorkerConnectionInfo connectionInfo; - private final NodeResourceManagerService nodeResourceManagerService; + private final NodeResourceManager nodeResourceManager; private AsyncRpcClient rmClient; private ServiceTracker serviceTracker; private TajoResourceTrackerProtocolService.Interface resourceTracker; private int queueingLimit; - public NodeStatusUpdaterService(WorkerConnectionInfo connectionInfo, NodeResourceManagerService resourceManagerService) { - super(NodeStatusUpdaterService.class.getSimpleName()); + public NodeStatusUpdater(WorkerConnectionInfo connectionInfo, NodeResourceManager resourceManager) { + super(NodeStatusUpdater.class.getSimpleName()); this.connectionInfo = connectionInfo; - this.nodeResourceManagerService = resourceManagerService; + this.nodeResourceManager = resourceManager; } @Override @@ -78,7 +78,7 @@ public void serviceInit(Configuration conf) throws Exception { this.tajoConf = (TajoConf) conf; this.heartBeatRequestQueue = Queues.newLinkedBlockingQueue(); this.serviceTracker = ServiceTrackerFactory.get(tajoConf); - this.nodeResourceManagerService.getDispatcher().register(NodeStatusEvent.EventType.class, this); + this.nodeResourceManager.getDispatcher().register(NodeStatusEvent.EventType.class, this); this.heartBeatInterval = tajoConf.getIntVar(TajoConf.ConfVars.WORKER_HEARTBEAT_INTERVAL); this.updaterThread = new StatusUpdaterThread(); super.serviceInit(conf); @@ -87,7 +87,7 @@ public void serviceInit(Configuration conf) throws Exception { @Override public void serviceStart() throws Exception { // if resource changed over than 50%, send reports - this.queueingLimit = nodeResourceManagerService.getTotalResource().getVirtualCores() / 2; + this.queueingLimit = nodeResourceManager.getTotalResource().getVirtualCores() / 2; updaterThread.start(); super.serviceStart(); @@ -140,8 +140,8 @@ private NodeHeartbeatRequestProto createHeartBeatReport() { private NodeHeartbeatRequestProto createNodeStatusReport() { NodeHeartbeatRequestProto.Builder requestProto = NodeHeartbeatRequestProto.newBuilder(); - requestProto.setTotalResource(nodeResourceManagerService.getTotalResource().getProto()); - requestProto.setAvailableResource(nodeResourceManagerService.getAvailableResource().getProto()); + requestProto.setTotalResource(nodeResourceManager.getTotalResource().getProto()); + requestProto.setAvailableResource(nodeResourceManager.getAvailableResource().getProto()); requestProto.setWorkerId(connectionInfo.getId()); requestProto.setConnectionInfo(connectionInfo.getProto()); diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeStatusUpdaterService.java b/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeStatusUpdater.java similarity index 92% rename from tajo-core/src/test/java/org/apache/tajo/worker/MockNodeStatusUpdaterService.java rename to tajo-core/src/test/java/org/apache/tajo/worker/MockNodeStatusUpdater.java index 934f29259c..2d7d0be6d8 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeStatusUpdaterService.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeStatusUpdater.java @@ -32,16 +32,16 @@ import static org.apache.tajo.ipc.TajoResourceTrackerProtocol.*; -public class MockNodeStatusUpdaterService extends NodeStatusUpdaterService { +public class MockNodeStatusUpdater extends NodeStatusUpdater { private CountDownLatch barrier; private Map membership = Maps.newHashMap(); private Map resources = Maps.newHashMap(); private MockResourceTracker resourceTracker; - public MockNodeStatusUpdaterService(CountDownLatch barrier, WorkerConnectionInfo connectionInfo, - NodeResourceManagerService resourceManagerService) { - super(connectionInfo, resourceManagerService); + public MockNodeStatusUpdater(CountDownLatch barrier, WorkerConnectionInfo connectionInfo, + NodeResourceManager resourceManager) { + super(connectionInfo, resourceManager); this.barrier = barrier; this.resourceTracker = new MockResourceTracker(); } diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManagerService.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java similarity index 85% rename from tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManagerService.java rename to tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java index 7586f37ec2..7407acc1f9 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManagerService.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java @@ -40,10 +40,10 @@ import static org.junit.Assert.*; import static org.apache.tajo.ipc.TajoWorkerProtocol.*; -public class TestNodeResourceManagerService { +public class TestNodeResourceManager { - private NodeResourceManagerService resourceManagerService; - private MockNodeStatusUpdaterService statusUpdater; + private NodeResourceManager resourceManager; + private MockNodeStatusUpdater statusUpdater; private AsyncDispatcher dispatcher; private int taskMemory; private TajoConf conf; @@ -64,19 +64,19 @@ public void setup() { dispatcher.init(conf); dispatcher.start(); - resourceManagerService = new NodeResourceManagerService(dispatcher); - resourceManagerService.init(conf); - resourceManagerService.start(); + resourceManager = new NodeResourceManager(dispatcher); + resourceManager.init(conf); + resourceManager.start(); WorkerConnectionInfo worker = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); - statusUpdater = new MockNodeStatusUpdaterService(new CountDownLatch(0), worker, resourceManagerService); + statusUpdater = new MockNodeStatusUpdater(new CountDownLatch(0), worker, resourceManager); statusUpdater.init(conf); statusUpdater.start(); } @After public void tearDown() { - resourceManagerService.stop(); + resourceManager.stop(); statusUpdater.stop(); dispatcher.stop(); } @@ -90,15 +90,15 @@ public void testNodeResourceAllocateEvent() throws Exception { ExecutionBlockId ebId = new ExecutionBlockId(LocalTajoTestingUtility.newQueryId(), 0); requestProto.setExecutionBlockId(ebId.getProto()); - assertEquals(resourceManagerService.getTotalResource(), resourceManagerService.getAvailableResource()); + assertEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); requestProto.addAllTaskRequest(createTaskRequests(taskMemory, requestSize)); dispatcher.getEventHandler().handle(new NodeResourceAllocateEvent(requestProto.build(), callFuture)); BatchAllocationResponseProto responseProto = callFuture.get(); - assertNotEquals(resourceManagerService.getTotalResource(), resourceManagerService.getAvailableResource()); + assertNotEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); assertEquals(0, responseProto.getCancellationTaskCount()); - assertEquals(requestSize, resourceManagerService.getAllocatedSize()); + assertEquals(requestSize, resourceManager.getAllocatedSize()); } @@ -112,14 +112,14 @@ public void testNodeResourceCancellation() throws Exception { ExecutionBlockId ebId = new ExecutionBlockId(LocalTajoTestingUtility.newQueryId(), 0); requestProto.setExecutionBlockId(ebId.getProto()); - assertEquals(resourceManagerService.getTotalResource(), resourceManagerService.getAvailableResource()); + assertEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); requestProto.addAllTaskRequest(createTaskRequests(taskMemory, requestSize + overSize)); dispatcher.getEventHandler().handle(new NodeResourceAllocateEvent(requestProto.build(), callFuture)); BatchAllocationResponseProto responseProto = callFuture.get(); assertEquals(overSize, responseProto.getCancellationTaskCount()); - assertEquals(requestSize, resourceManagerService.getAllocatedSize()); + assertEquals(requestSize, resourceManager.getAllocatedSize()); } @Test @@ -131,23 +131,23 @@ public void testNodeResourceDeallocateEvent() throws Exception { ExecutionBlockId ebId = new ExecutionBlockId(LocalTajoTestingUtility.newQueryId(), 0); requestProto.setExecutionBlockId(ebId.getProto()); - assertEquals(resourceManagerService.getTotalResource(), resourceManagerService.getAvailableResource()); + assertEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); requestProto.addAllTaskRequest(createTaskRequests(taskMemory, requestSize)); dispatcher.getEventHandler().handle(new NodeResourceAllocateEvent(requestProto.build(), callFuture)); BatchAllocationResponseProto responseProto = callFuture.get(); - assertNotEquals(resourceManagerService.getTotalResource(), resourceManagerService.getAvailableResource()); + assertNotEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); assertEquals(0, responseProto.getCancellationTaskCount()); - assertEquals(requestSize, resourceManagerService.getAllocatedSize()); + assertEquals(requestSize, resourceManager.getAllocatedSize()); //deallocate for(TaskAllocationRequestProto allocationRequestProto : requestProto.getTaskRequestList()) { // direct invoke handler for testing - resourceManagerService.handle(new NodeResourceDeallocateEvent(allocationRequestProto.getResource())); + resourceManager.handle(new NodeResourceDeallocateEvent(allocationRequestProto.getResource())); } - assertEquals(0, resourceManagerService.getAllocatedSize()); - assertEquals(resourceManagerService.getTotalResource(), resourceManagerService.getAvailableResource()); + assertEquals(0, resourceManager.getAllocatedSize()); + assertEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); } @Test(timeout = 30000) diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdaterService.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java similarity index 77% rename from tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdaterService.java rename to tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java index ae0fdda2eb..fb3c77e27c 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdaterService.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java @@ -31,10 +31,10 @@ import java.util.concurrent.CountDownLatch; import static org.junit.Assert.*; -public class TestNodeStatusUpdaterService { +public class TestNodeStatusUpdater { - private NodeResourceManagerService resourceManagerService; - private MockNodeStatusUpdaterService statusUpdater; + private NodeResourceManager resourceManager; + private MockNodeStatusUpdater statusUpdater; private AsyncDispatcher dispatcher; private TajoConf conf; @@ -48,14 +48,14 @@ public void setup() { dispatcher.init(conf); dispatcher.start(); - resourceManagerService = new NodeResourceManagerService(dispatcher); - resourceManagerService.init(conf); - resourceManagerService.start(); + resourceManager = new NodeResourceManager(dispatcher); + resourceManager.init(conf); + resourceManager.start(); } @After public void tearDown() { - resourceManagerService.stop(); + resourceManager.stop(); if (statusUpdater != null) statusUpdater.stop(); dispatcher.stop(); } @@ -64,18 +64,18 @@ public void tearDown() { public void testNodeMembership() throws Exception { CountDownLatch barrier = new CountDownLatch(1); WorkerConnectionInfo worker = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); - statusUpdater = new MockNodeStatusUpdaterService(barrier, worker, resourceManagerService); + statusUpdater = new MockNodeStatusUpdater(barrier, worker, resourceManager); statusUpdater.init(conf); statusUpdater.start(); - MockNodeStatusUpdaterService.MockResourceTracker resourceTracker = statusUpdater.getResourceTracker(); + MockNodeStatusUpdater.MockResourceTracker resourceTracker = statusUpdater.getResourceTracker(); barrier.await(); assertTrue(resourceTracker.getTotalResource().containsKey(worker.getId())); - assertEquals(resourceManagerService.getTotalResource(), + assertEquals(resourceManager.getTotalResource(), resourceTracker.getTotalResource().get(worker.getId())); - assertEquals(resourceManagerService.getAvailableResource(), + assertEquals(resourceManager.getAvailableResource(), resourceTracker.getAvailableResource().get(worker.getId())); } @@ -83,11 +83,11 @@ public void testNodeMembership() throws Exception { public void testPing() throws Exception { CountDownLatch barrier = new CountDownLatch(2); WorkerConnectionInfo worker = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); - statusUpdater = new MockNodeStatusUpdaterService(barrier, worker, resourceManagerService); + statusUpdater = new MockNodeStatusUpdater(barrier, worker, resourceManager); statusUpdater.init(conf); statusUpdater.start(); - MockNodeStatusUpdaterService.MockResourceTracker resourceTracker = statusUpdater.getResourceTracker(); + MockNodeStatusUpdater.MockResourceTracker resourceTracker = statusUpdater.getResourceTracker(); barrier.await(); TajoResourceTrackerProtocol.NodeHeartbeatRequestProto lastRequest = resourceTracker.getLastRequest(); @@ -101,13 +101,13 @@ public void testPing() throws Exception { public void testResourceReport() throws Exception { CountDownLatch barrier = new CountDownLatch(2); WorkerConnectionInfo worker = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); - statusUpdater = new MockNodeStatusUpdaterService(barrier, worker, resourceManagerService); + statusUpdater = new MockNodeStatusUpdater(barrier, worker, resourceManager); statusUpdater.init(conf); statusUpdater.start(); for (int i = 0; i < statusUpdater.getQueueingLimit(); i++) { dispatcher.getEventHandler().handle(new NodeStatusEvent(NodeStatusEvent.EventType.REPORT_RESOURCE, - resourceManagerService.getAvailableResource())); + resourceManager.getAvailableResource())); } barrier.await(); assertEquals(0, statusUpdater.getQueueSize()); From ebd1a53dcf2c1599a4ba346bf8422103dc49f223 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Thu, 28 May 2015 14:53:39 +0900 Subject: [PATCH 15/21] TAJO-1615 --- .../physical/HashShuffleFileWriteExec.java | 2 +- .../engine/planner/physical/PhysicalExec.java | 2 +- .../tajo/worker/ExecutionBlockContext.java | 14 +- .../{TaskImpl.java => LegacyTaskImpl.java} | 82 +++--- .../tajo/worker/NodeResourceManager.java | 10 +- .../java/org/apache/tajo/worker/Task.java | 25 +- .../tajo/worker/TaskAttemptContext.java | 57 ++--- .../org/apache/tajo/worker/TaskContainer.java | 65 ++--- .../org/apache/tajo/worker/TaskExecutor.java | 242 ++++++++++++++---- .../org/apache/tajo/worker/TaskManager.java | 40 +-- .../org/apache/tajo/worker/TaskRunner.java | 7 +- .../src/main/proto/TajoWorkerProtocol.proto | 1 + .../tajo/querymaster/TestKillQuery.java | 11 +- .../org/apache/tajo/worker/TestFetcher.java | 14 +- .../tajo/worker/TestNodeResourceManager.java | 19 +- .../tajo/worker/TestNodeStatusUpdater.java | 2 +- 16 files changed, 366 insertions(+), 227 deletions(-) rename tajo-core/src/main/java/org/apache/tajo/worker/{TaskImpl.java => LegacyTaskImpl.java} (94%) diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashShuffleFileWriteExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashShuffleFileWriteExec.java index f1e2fe5060..1a92a7a5f6 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashShuffleFileWriteExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashShuffleFileWriteExec.java @@ -86,7 +86,7 @@ private HashShuffleAppender getAppender(int partId) throws IOException { HashShuffleAppender appender = appenderMap.get(partId); if (appender == null) { appender = hashShuffleAppenderManager.getAppender(context.getConf(), - context.getQueryId().getTaskId().getExecutionBlockId(), partId, meta, outSchema); + context.getTaskId().getTaskId().getExecutionBlockId(), partId, meta, outSchema); appenderMap.put(partId, appender); } return appender; diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalExec.java index de14c9a9d7..87a19a9f1d 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalExec.java @@ -79,7 +79,7 @@ protected void fatal(Log log, String message) { } protected Path getExecutorTmpDir() { - return new Path(context.getQueryId().getTaskId().getExecutionBlockId().getQueryId().toString(), + return new Path(context.getTaskId().getTaskId().getExecutionBlockId().getQueryId().toString(), UUID.randomUUID().toString()); } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java b/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java index e97955311e..187c01fc00 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java @@ -183,9 +183,9 @@ public void stop(){ // If ExecutionBlock is stopped, all running or pending tasks will be marked as failed. for (Task task : tasks.values()) { - if (task.getStatus() == TajoProtos.TaskAttemptState.TA_PENDING || - task.getStatus() == TajoProtos.TaskAttemptState.TA_RUNNING) { - task.setState(TajoProtos.TaskAttemptState.TA_FAILED); + if (task.getTaskContext().getState() == TajoProtos.TaskAttemptState.TA_PENDING || + task.getTaskContext().getState() == TajoProtos.TaskAttemptState.TA_RUNNING) { + try{ task.abort(); } catch (Throwable e){ @@ -395,13 +395,11 @@ public void run() { } else { for (Task task : new ArrayList(tasks.values())){ - if (task.isRunning() && task.isProgressChanged()) { - task.updateProgress(); + if (task.getTaskContext().getState() == + TajoProtos.TaskAttemptState.TA_RUNNING && task.isProgressChanged()) { masterStub.statusUpdate(null, task.getReport(), NullCallback.get()); - task.getContext().setProgressChanged(false); - } else { - task.updateProgress(); } + task.updateProgress(); } } } catch (Throwable t) { diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java b/tajo-core/src/main/java/org/apache/tajo/worker/LegacyTaskImpl.java similarity index 94% rename from tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java rename to tajo-core/src/main/java/org/apache/tajo/worker/LegacyTaskImpl.java index 28c478790e..9fd8b61d36 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/LegacyTaskImpl.java @@ -68,14 +68,13 @@ import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto; import static org.apache.tajo.plan.serder.PlanProto.ShuffleType; -public class TaskImpl implements Task { - private static final Log LOG = LogFactory.getLog(TaskImpl.class); +public class LegacyTaskImpl implements Task { + private static final Log LOG = LogFactory.getLog(LegacyTaskImpl.class); private static final float FETCHER_PROGRESS = 0.5f; private final TajoConf systemConf; private final QueryContext queryContext; private final ExecutionBlockContext executionBlockContext; - private final TaskAttemptId taskId; private final String taskRunnerId; private final Path taskDir; @@ -99,23 +98,22 @@ public class TaskImpl implements Task { private Schema finalSchema = null; private TupleComparator sortComp = null; - public TaskImpl(String taskRunnerId, - Path baseDir, - TaskAttemptId taskId, - final ExecutionBlockContext executionBlockContext, - final TaskRequest request) throws IOException { + public LegacyTaskImpl(String taskRunnerId, + Path baseDir, + TaskAttemptId taskId, + final ExecutionBlockContext executionBlockContext, + final TaskRequest request) throws IOException { this(taskRunnerId, baseDir, taskId, executionBlockContext.getConf(), executionBlockContext, request); } - public TaskImpl(String taskRunnerId, - Path baseDir, - TaskAttemptId taskId, - TajoConf conf, - final ExecutionBlockContext executionBlockContext, - final TaskRequest request) throws IOException { + public LegacyTaskImpl(String taskRunnerId, + Path baseDir, + TaskAttemptId taskId, + TajoConf conf, + final ExecutionBlockContext executionBlockContext, + final TaskRequest request) throws IOException { this.taskRunnerId = taskRunnerId; this.request = request; - this.taskId = taskId; this.systemConf = conf; this.queryContext = request.getQueryContext(systemConf); @@ -162,7 +160,7 @@ public void initPlan() throws IOException { } } else { Path outFilePath = ((FileTablespace) TableSpaceManager.getFileStorageManager(systemConf)) - .getAppenderFilePath(taskId, queryContext.getStagingDir()); + .getAppenderFilePath(getId(), queryContext.getStagingDir()); LOG.info("Output File Path: " + outFilePath); context.setOutputPath(outFilePath); } @@ -200,6 +198,7 @@ private void stopScriptExecutors() { } } + @Override public void init() throws IOException { initPlan(); startScriptExecutors(); @@ -228,38 +227,32 @@ public void init() throws IOException { } } - public TaskAttemptId getTaskId() { - return taskId; - } - - @Override - public TaskAttemptId getId() { + private TaskAttemptId getId() { return context.getTaskId(); } - public TaskAttemptState getStatus() { - return context.getState(); - } - public String toString() { - return "queryId: " + this.getId() + " status: " + this.getStatus(); + return "queryId: " + this.getId() + " status: " + context.getState(); } - public void setState(TaskAttemptState status) { - context.setState(status); + @Override + public boolean isStopped() { + return context.isStopped(); } - public TaskAttemptContext getContext() { + @Override + public TaskAttemptContext getTaskContext() { return context; } @Override - public boolean hasFetchPhase() { - return fetcherRunners.size() > 0; + public ExecutionBlockContext getExecutionBlockContext() { + return executionBlockContext; } - public List getFetchers() { - return new ArrayList(fetcherRunners); + @Override + public boolean hasFetchPhase() { + return fetcherRunners.size() > 0; } @Override @@ -277,11 +270,14 @@ public void kill() { context.stop(); } + @Override public void abort() { stopScriptExecutors(); + context.setState(TajoProtos.TaskAttemptState.TA_FAILED); context.stop(); } + @Override public TaskStatusProto getReport() { TaskStatusProto.Builder builder = TaskStatusProto.newBuilder(); builder.setWorkerName(executionBlockContext.getWorkerContext().getConnectionInfo().getHostAndPeerRpcPort()); @@ -297,14 +293,12 @@ public TaskStatusProto getReport() { return builder.build(); } - public boolean isRunning(){ - return context.getState() == TaskAttemptState.TA_RUNNING; - } - + @Override public boolean isProgressChanged() { return context.isProgressChanged(); } + @Override public void updateProgress() { if(context != null && context.isStopped()){ return; @@ -403,7 +397,6 @@ public void run() throws Exception { // complete. waitForFetch(); context.setFetcherProgress(FETCHER_PROGRESS); - context.setProgressChanged(true); updateProgress(); } @@ -431,7 +424,7 @@ public void run() throws Exception { } executionBlockContext.completedTasksNum.incrementAndGet(); - context.getHashShuffleAppenderManager().finalizeTask(taskId); + context.getHashShuffleAppenderManager().finalizeTask(getId()); QueryMasterProtocol.QueryMasterProtocolService.Interface queryMasterStub = executionBlockContext.getStub(); if (context.isStopped()) { @@ -459,6 +452,7 @@ public void run() throws Exception { } } else { // if successful + context.stop(); context.setProgress(1.0f); context.setState(TaskAttemptState.TA_SUCCEEDED); executionBlockContext.succeededTasksNum.incrementAndGet(); @@ -472,7 +466,6 @@ public void run() throws Exception { ", succeeded: " + executionBlockContext.succeededTasksNum.intValue() + ", killed: " + executionBlockContext.killedTasksNum.intValue() + ", failed: " + executionBlockContext.failedTasksNum.intValue()); - cleanup(); } } @@ -500,7 +493,7 @@ public void cleanup() { public TaskHistory createTaskHistory() { TaskHistory taskHistory = null; try { - taskHistory = new TaskHistory(getTaskId(), getStatus(), context.getProgress(), + taskHistory = new TaskHistory(context.getTaskId(), context.getState(), context.getProgress(), startTime, finishTime, reloadInputStats()); if (context.getOutputPath() != null) { @@ -546,8 +539,8 @@ public int hashCode() { } public boolean equals(Object obj) { - if (obj instanceof TaskImpl) { - TaskImpl other = (TaskImpl) obj; + if (obj instanceof LegacyTaskImpl) { + LegacyTaskImpl other = (LegacyTaskImpl) obj; return this.context.equals(other.context); } return false; @@ -668,7 +661,6 @@ private synchronized void fetcherFinished(TaskAttemptContext ctx) { context.setFetcherProgress(FETCHER_PROGRESS); } else { context.setFetcherProgress(adjustFetchProcess(fetcherSize, remainFetcher)); - context.setProgressChanged(true); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java b/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java index 18e840fc97..c71f03ebfa 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java @@ -46,7 +46,7 @@ public class NodeResourceManager extends AbstractService implements EventHandler private AtomicInteger allocatedSize; private TajoConf tajoConf; - public NodeResourceManager(Dispatcher dispatcher, TaskManager taskManager){ + public NodeResourceManager(Dispatcher dispatcher, TaskManager taskManager) { super(NodeResourceManager.class.getName()); this.dispatcher = dispatcher; this.taskManager = taskManager; @@ -76,8 +76,14 @@ public void handle(NodeResourceManagerEvent event) { NodeResource resource = new NodeResource(request.getResource()); if (allocate(resource)) { allocatedSize.incrementAndGet(); + if(allocateEvent.getRequest().hasExecutionBlockRequest()){ + //send ExecutionBlock start event to TaskManager + taskManager.getEventHandler().handle( + new ExecutionBlockStartEvent(allocateEvent.getRequest().getExecutionBlockRequest())); + } + //send task start event to TaskExecutor - //taskManager.getEventHandler().handle(new TaskStartEvent(request.getTaskRequest(), resource)); + taskManager.getEventHandler().handle(new TaskStartEvent(request.getTaskRequest(), resource)); } else { // reject the exceeded requests response.addCancellationTask(request); diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java index f67782c9ec..c84994003e 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java @@ -18,26 +18,35 @@ package org.apache.tajo.worker; -import org.apache.tajo.TajoProtos; -import org.apache.tajo.TaskAttemptId; +import org.apache.tajo.ipc.TajoWorkerProtocol; import java.io.IOException; public interface Task { - TaskAttemptId getId(); - - TajoProtos.TaskAttemptState getStatus(); - void init() throws IOException; - boolean hasFetchPhase(); - void fetch(); void run() throws Exception; void kill(); + void abort(); + void cleanup(); + + boolean hasFetchPhase(); + + boolean isProgressChanged(); + + boolean isStopped(); + + void updateProgress(); + + TaskAttemptContext getTaskContext(); + + ExecutionBlockContext getExecutionBlockContext(); + + TajoWorkerProtocol.TaskStatusProto getReport(); } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java index 58028ac014..de0bd9b19b 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java @@ -60,13 +60,13 @@ public class TaskAttemptContext { private volatile TaskAttemptState state; private TableStats resultStats; - private TaskAttemptId queryId; + private TaskAttemptId taskId; private final Path workDir; private boolean needFetch = false; private CountDownLatch doneFetchPhaseSignal; private float progress = 0.0f; private float fetcherProgress = 0.0f; - private AtomicBoolean progressChanged = new AtomicBoolean(false); + private volatile boolean progressChanged; /** a map of shuffled file outputs */ private Map shuffleFileOutputs; @@ -87,7 +87,7 @@ public class TaskAttemptContext { private EvalContext evalContext = new EvalContext(); public TaskAttemptContext(QueryContext queryContext, final ExecutionBlockContext executionBlockContext, - final TaskAttemptId queryId, + final TaskAttemptId taskId, final FragmentProto[] fragments, final Path workDir) { this.queryContext = queryContext; @@ -97,7 +97,7 @@ public TaskAttemptContext(QueryContext queryContext, final ExecutionBlockContext this.sharedResource = executionBlockContext.getSharedResource(); } - this.queryId = queryId; + this.taskId = taskId; if (fragments != null) { for (FragmentProto t : fragments) { @@ -114,25 +114,15 @@ public TaskAttemptContext(QueryContext queryContext, final ExecutionBlockContext this.workDir = workDir; this.shuffleFileOutputs = Maps.newHashMap(); - state = TaskAttemptState.TA_PENDING; + this.state = TaskAttemptState.TA_PENDING; this.partitionOutputVolume = Maps.newHashMap(); - - if (workerContext != null) { - this.hashShuffleAppenderManager = workerContext.getHashShuffleAppenderManager(); - } else { - try { - this.hashShuffleAppenderManager = new HashShuffleAppenderManager(queryContext.getConf()); - } catch (IOException e) { - LOG.error(e.getMessage(), e); - } - } } @VisibleForTesting - public TaskAttemptContext(final QueryContext queryContext, final TaskAttemptId queryId, + public TaskAttemptContext(final QueryContext queryContext, final TaskAttemptId taskAttemptId, final Fragment [] fragments, final Path workDir) { - this(queryContext, null, queryId, FragmentConvertor.toFragmentProtoArray(fragments), workDir); + this(queryContext, null, taskAttemptId, FragmentConvertor.toFragmentProtoArray(fragments), workDir); } public TajoConf getConf() { @@ -310,7 +300,7 @@ public Path getWorkDir() { } public TaskAttemptId getTaskId() { - return this.queryId; + return this.taskId; } public float getProgress() { @@ -326,17 +316,11 @@ public void setProgress(float progress) { this.progress = progress; } - if (previousProgress != progress) { - setProgressChanged(true); - } + this.progressChanged = previousProgress != progress; } public boolean isProgressChanged() { - return progressChanged.get(); - } - - public void setProgressChanged(boolean changed){ - progressChanged.set(changed); + return progressChanged; } public void setExecutorProgress(float executorProgress) { @@ -355,7 +339,9 @@ public void setFetcherProgress(float fetcherProgress) { if(Float.isNaN(fetcherProgress) || Float.isInfinite(fetcherProgress)){ fetcherProgress = 0.0f; } + float previousProgress = this.fetcherProgress; this.fetcherProgress = fetcherProgress; + this.progressChanged = previousProgress != fetcherProgress; } public FragmentProto getTable(String id) { @@ -383,13 +369,13 @@ public Collection getInputTables() { } public int hashCode() { - return Objects.hashCode(queryId); + return Objects.hashCode(taskId); } public boolean equals(Object obj) { if (obj instanceof TaskAttemptContext) { TaskAttemptContext other = (TaskAttemptContext) obj; - return queryId.equals(other.getTaskId()); + return taskId.equals(other.getTaskId()); } else { return false; } @@ -399,11 +385,18 @@ public QueryContext getQueryContext() { return queryContext; } - public TaskAttemptId getQueryId() { - return queryId; - } - public HashShuffleAppenderManager getHashShuffleAppenderManager() { + if(hashShuffleAppenderManager == null) { + if (workerContext != null) { + this.hashShuffleAppenderManager = workerContext.getHashShuffleAppenderManager(); + } else { + try { + this.hashShuffleAppenderManager = new HashShuffleAppenderManager(queryContext.getConf()); + } catch (IOException e) { + LOG.error(e.getMessage(), e); + } + } + } return hashShuffleAppenderManager; } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskContainer.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskContainer.java index e6fd62dd2e..fd083d1c3c 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskContainer.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskContainer.java @@ -48,50 +48,51 @@ public class TaskContainer implements Runnable { private static final Log LOG = LogFactory.getLog(TaskContainer.class); // Contains the object references related for TaskRunner - private ExecutionBlockContext executionBlockContext; - private Task task = null; + private final TaskExecutor executor; + private final int sequenceId; - public TaskContainer(ExecutionBlockContext executionBlockContext, Task task) { - this.executionBlockContext = executionBlockContext; - this.task = task; + public TaskContainer(int sequenceId, TaskExecutor executor) { + this.sequenceId = sequenceId; + this.executor = executor; } public void init() throws IOException { - if (executionBlockContext.isStopped()) return; + //if (executionBlockContext.isStopped()) return; - LOG.info("Initializing: " + task.getId()); - getContext().getWorkerContext().getWorkerSystemMetrics().counter("query", "task").inc(); - task.init(); +// LOG.info("Initializing: " + task.getId()); +// getContext().getWorkerContext().getWorkerSystemMetrics().counter("query", "task").inc(); } - public void stop() { - task.cleanupTask(); - } + @Override + public void run() { + while (true) { + Task task = null; + try { + task = executor.getNextTask(); + LOG.debug(sequenceId + " got task:" + task.getTaskContext().getTaskId()); - public Task getTask() { - return task; - } + TaskAttemptContext taskAttemptContext = task.getTaskContext(); + task.init(); - public ExecutionBlockContext getContext() { - return executionBlockContext; - } + if (taskAttemptContext.isStopped()) return; - @Override - public void run() { - if (executionBlockContext.isStopped()) return; + if (task.hasFetchPhase()) { + task.fetch(); // The fetch is performed in an asynchronous way. + } - try { - if (task.hasFetchPhase()) { - task.fetch(); // The fetch is performed in an asynchronous way. - } - if (!executionBlockContext.isStopped()) { - task.run(); + if (!taskAttemptContext.isStopped()) { + task.run(); + } + + task.cleanup(); + } catch (Exception t) { + LOG.error(t.getMessage(), t); + if(task != null){ + task.getExecutionBlockContext().fatalError(task.getTaskContext().getTaskId(), t.getMessage()); + } + } finally { + executor.stopTask(task); } - } catch (Exception t) { - LOG.error(t.getMessage(), t); - getContext().fatalError(task.getId(), t.getMessage()); - } finally { - task.cleanupTask(); } } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java index 701f860420..075fadac60 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java @@ -22,94 +22,235 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.service.AbstractService; import org.apache.hadoop.yarn.event.EventHandler; import org.apache.tajo.ExecutionBlockId; +import org.apache.tajo.TajoProtos; import org.apache.tajo.TaskAttemptId; import org.apache.tajo.TaskId; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.conf.TajoConf.ConfVars; +import org.apache.tajo.engine.query.TaskRequest; +import org.apache.tajo.ipc.TajoWorkerProtocol; import org.apache.tajo.resource.NodeResource; -import org.apache.tajo.worker.event.NodeResourceDeallocateEvent; -import org.apache.tajo.worker.event.TaskExecutorEvent; -import org.apache.tajo.worker.event.TaskManagerEvent; -import org.apache.tajo.worker.event.TaskStartEvent; +import org.apache.tajo.resource.NodeResources; +import org.apache.tajo.worker.event.*; import java.io.IOException; import java.util.Map; import java.util.concurrent.*; +import java.util.concurrent.atomic.AtomicInteger; -public class TaskExecutor extends ThreadPoolExecutor +public class TaskExecutor extends AbstractService implements EventHandler { private static final Log LOG = LogFactory.getLog(TaskExecutor.class); - private final TajoConf conf; private final TaskManager taskManager; - private final NodeResourceManager resourceManager; + private final EventHandler rmEventHandler; private final Map allocatedResourceMap; - private final ThreadPoolExecutor fetcherExecutor; + private final BlockingQueue taskQueue; + private final AtomicInteger runningTasks; + private ThreadPoolExecutor fetcherExecutor; + private ExecutorService threadPool; + private TajoConf tajoConf; - public TaskExecutor(TajoConf conf, TaskManager taskManager, NodeResourceManager resourceManager) { - this(conf, conf.getIntVar(ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES), taskManager, resourceManager); + public TaskExecutor(TaskManager taskManager, EventHandler rmEventHandler) { + super(TaskExecutor.class.getName()); + this.taskManager = taskManager; + this.rmEventHandler = rmEventHandler; + this.allocatedResourceMap = Maps.newConcurrentMap(); + this.runningTasks = new AtomicInteger(); + this.taskQueue = new LinkedBlockingQueue(); + } + @Override + protected void serviceInit(Configuration conf) throws Exception { + if (!(conf instanceof TajoConf)) { + throw new IllegalArgumentException("Configuration must be a TajoConf instance"); + } + this.tajoConf = (TajoConf) conf; + super.serviceInit(conf); } - public TaskExecutor(TajoConf conf, int nThreads, TaskManager taskManager, NodeResourceManager resourceManager) { - super(nThreads, nThreads, - 0L, TimeUnit.MILLISECONDS, - new LinkedBlockingQueue(), + @Override + protected void serviceStart() throws Exception { + int nThreads = this.tajoConf.getIntVar(ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES); + this.threadPool = Executors.newFixedThreadPool(nThreads, new ThreadFactoryBuilder().setNameFormat("Task executor #%d").build()); - this.conf = conf; - this.taskManager = taskManager; - this.resourceManager = resourceManager; - this.allocatedResourceMap = Maps.newHashMap(); - int maxFetcherThreads = conf.getIntVar(ConfVars.SHUFFLE_FETCHER_PARALLEL_EXECUTION_MAX_NUM); + int maxFetcherThreads = tajoConf.getIntVar(ConfVars.SHUFFLE_FETCHER_PARALLEL_EXECUTION_MAX_NUM); this.fetcherExecutor = new ThreadPoolExecutor(Math.min(nThreads, maxFetcherThreads), maxFetcherThreads, 60L, TimeUnit.SECONDS, new SynchronousQueue(true)); - LOG.info("Startup TaskExecutor"); + + + for (int i = 0; i < nThreads; i++) { + threadPool.submit(new TaskContainer(i, this)); + } + + super.serviceStart(); + LOG.info("Started TaskExecutor[" + nThreads + "], Fetcher executor[" + maxFetcherThreads + "]"); } - @Override - protected void beforeExecute(Thread t, Runnable r) { - TaskContainer container = (T) r; + /** + * This will block until a task is available. + */ + protected Task getNextTask() { + Task task = null; try { - container.init(); - } catch (IOException e) { - LOG.error(e.getMessage(), e); - container.getContext().fatalError(container.getTask().getId(), e.getMessage()); + task = taskQueue.take(); + } catch (InterruptedException e) { + LOG.fatal(e); } + return task; } - @Override + + protected void beforeExecute(Thread t, Runnable r) { + System.out.println(r); + Future future = (Future) r; + + TaskContainer container = null; +// try { +// container = future.get(); +// } catch (InterruptedException e) { +// e.printStackTrace(); +// } catch (ExecutionException e) { +// e.printStackTrace(); +// } +// try { +// container.init(); +// } catch (IOException e) { +// LOG.error(e.getMessage(), e); +// container.getTask().abort(); +// container.getContext().fatalError(container.getTask().getId(), e.getMessage()); + //} + } + + protected void afterExecute(Runnable r, Throwable t) { - TaskContainer container = (T) r; + TaskContainer container = null; + try { + container = ((Future) r).get(); + } catch (InterruptedException e) { + e.printStackTrace(); + } catch (ExecutionException e) { + e.printStackTrace(); + } try { - if (container != null) { + if (t != null) { LOG.error(t.getMessage(), t); - container.getContext().fatalError(container.getTask().getId(), t.getMessage()); + // container.getContext().fatalError(container.getTask().getId(), t.getMessage()); return; } try { - container.stop(); + // container.stop(); } catch (Throwable throwable) { - LOG.error(t.getMessage(), t); - container.getContext().fatalError(container.getTask().getId(), t.getMessage()); + LOG.error(throwable.getMessage(), throwable); + //container.getContext().fatalError(container.getTask().getId(), throwable.getMessage()); } } finally { - //FIXME change to TaskStopEvent ? - resourceManager.getDispatcher().getEventHandler() - .handle(new NodeResourceDeallocateEvent(allocatedResourceMap.remove(container.getTask().getId()))); + } } + @SuppressWarnings("unchecked") + protected void stopTask(Task task) { + //FIXME change to TaskStopEvent ? + if(task != null) { + rmEventHandler + .handle(new NodeResourceDeallocateEvent(allocatedResourceMap.remove(task.getTaskContext().getTaskId()))); + runningTasks.decrementAndGet(); + } + + } protected ExecutorService getFetcherExecutor() { return fetcherExecutor; } + /*FIXME*/ + protected Task createTask(final ExecutionBlockContext context, TajoWorkerProtocol.TaskRequestProto taskRequest){ + + final TaskAttemptId taskAttemptId = new TaskAttemptId(taskRequest.getId()); +// if (getContext().getTasks().containsKey(taskAttemptId)) { +// LOG.error("Duplicate Task Attempt: " + taskAttemptId); +// fatalError(qmClientService, taskAttemptId, "Duplicate Task Attempt: " + taskAttemptId); +// continue; +// } + final TaskAttemptContext taskAttemptContext = new TaskAttemptContext(null,context,taskAttemptId, null, null); + //LOG.info("Initializing: " + taskAttemptId); + return new Task() { + @Override + public void init() throws IOException { + + } + + @Override + public void fetch() { + + } + + @Override + public void run() throws Exception { + + } + + @Override + public void kill() { + + } + + @Override + public void abort() { + + } + + @Override + public void cleanup() { + + } + + @Override + public boolean hasFetchPhase() { + return false; + } + + @Override + public boolean isProgressChanged() { + return false; + } + + @Override + public boolean isStopped() { + return taskAttemptContext.isStopped(); + } + + @Override + public void updateProgress() { + + } + + @Override + public TaskAttemptContext getTaskContext() { + return taskAttemptContext; + } + + @Override + public ExecutionBlockContext getExecutionBlockContext() { + return context; + } + + @Override + public TajoWorkerProtocol.TaskStatusProto getReport() { + return null; + } + }; + } + @Override public void handle(TaskExecutorEvent event) { @@ -119,26 +260,15 @@ public void handle(TaskExecutorEvent event) { final TaskStartEvent startEvent = (TaskStartEvent)event; allocatedResourceMap.put(startEvent.getTaskId(), startEvent.getAllocatedResource()); -// FIXME - submit(new TaskContainer( - taskManager.getExecutionBlockContext( - startEvent.getTaskId().getTaskId().getExecutionBlockId()), new Task(){ - - @Override - public void waitForFetch() throws InterruptedException, IOException { - - } - - @Override - public void run() throws Exception { - - } + ExecutionBlockContext context = taskManager.getExecutionBlockContext( + startEvent.getTaskId().getTaskId().getExecutionBlockId()); - @Override - public TaskAttemptId getId() { - return startEvent.getTaskId(); - } - })); + try { + taskQueue.put(createTask(context, startEvent.getTaskRequest())); + runningTasks.incrementAndGet(); + } catch (InterruptedException e) { + LOG.fatal(e.getMessage(), e); + } } } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java index a7e24a3f7c..a87bd530ec 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java @@ -23,6 +23,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.service.AbstractService; import org.apache.hadoop.service.CompositeService; import org.apache.hadoop.yarn.event.Dispatcher; import org.apache.hadoop.yarn.event.EventHandler; @@ -45,24 +46,30 @@ public class TaskManager extends CompositeService implements EventHandler { private static final Log LOG = LogFactory.getLog(TaskManager.class); - private final NodeResourceManager resourceManager; private final TajoWorker.WorkerContext workerContext; private final Map executionBlockContextMap; private final Dispatcher dispatcher; - private final TaskExecutor taskExecutor; private TajoConf tajoConf; + private TaskExecutor taskExecutor; private volatile boolean isStopped = false; private TaskHistoryCleanerThread taskHistoryCleanerThread; + private EventHandler rmEventHandler; - public TaskManager(Dispatcher dispatcher, TajoWorker.WorkerContext workerContext, NodeResourceManager resourceManager) { + public TaskManager(Dispatcher dispatcher, TajoWorker.WorkerContext workerContext, + EventHandler rmEventHandler) { + this(dispatcher, workerContext, null, rmEventHandler); + } + + public TaskManager(Dispatcher dispatcher, TajoWorker.WorkerContext workerContext, TaskExecutor taskExecutor, + EventHandler rmEventHandler) { super(TaskManager.class.getName()); this.dispatcher = dispatcher; this.workerContext = workerContext; - this.resourceManager = resourceManager; this.executionBlockContextMap = Maps.newHashMap(); - this.taskExecutor = new TaskExecutor(workerContext.getConf(), this, resourceManager); + this.taskExecutor = taskExecutor; + this.rmEventHandler = rmEventHandler; } @Override @@ -70,12 +77,19 @@ protected void serviceInit(Configuration conf) throws Exception { if (!(conf instanceof TajoConf)) { throw new IllegalArgumentException("Configuration must be a TajoConf instance"); } - tajoConf = (TajoConf)conf; - addIfService(this.taskExecutor); + this.tajoConf = (TajoConf)conf; + + if (taskExecutor == null) { + this.taskExecutor = new TaskExecutor(this, rmEventHandler); + } + addIfService(dispatcher); + addService(taskExecutor); + + dispatcher.register(TaskExecutorEvent.EventType.class, taskExecutor); dispatcher.register(TaskManagerEvent.EventType.class, this); taskHistoryCleanerThread = new TaskHistoryCleanerThread(); - super.serviceInit(tajoConf); + super.serviceInit(conf); } @Override @@ -88,16 +102,10 @@ protected void serviceStart() throws Exception { protected void serviceStop() throws Exception { isStopped = true; -// synchronized(executionBlockContextMap) { -// for(TaskRunner eachTaskRunner: taskRunnerMap.values()) { -// if(!eachTaskRunner.isStopped()) { -// eachTaskRunner.stop(); -// } -// } -// } for(ExecutionBlockContext context: executionBlockContextMap.values()) { context.stop(); } + executionBlockContextMap.clear(); if(taskHistoryCleanerThread != null) { taskHistoryCleanerThread.interrupt(); @@ -136,8 +144,6 @@ public void handle(TaskManagerEvent event) { } executionBlockContextMap.put(startEvent.getExecutionBlockId(), context); } - //get NodeResourceManager().getEventHandler().handler() - } else if (event instanceof ExecutionBlockStopEvent) { ExecutionBlockStopEvent stopEvent = (ExecutionBlockStopEvent) event; ExecutionBlockContext executionBlockContext = executionBlockContextMap.remove(stopEvent.getExecutionBlockId()); diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunner.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunner.java index 57493bfd90..aff2daf62a 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunner.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunner.java @@ -34,10 +34,8 @@ import org.apache.tajo.master.container.TajoContainerIdPBImpl; import org.apache.tajo.master.container.TajoConverterUtils; import org.apache.tajo.rpc.CallFuture; -import org.apache.tajo.rpc.NettyClientBase; import org.apache.tajo.rpc.NullCallback; -import java.net.ConnectException; import java.util.concurrent.*; import static org.apache.tajo.ipc.TajoWorkerProtocol.*; @@ -256,7 +254,7 @@ public void run() { LOG.info("Initializing: " + taskAttemptId); Task task = null; try { - task = new TaskImpl(getId(), getTaskBaseDir(), taskAttemptId, executionBlockContext, + task = new LegacyTaskImpl(getId(), getTaskBaseDir(), taskAttemptId, executionBlockContext, new TaskRequestImpl(taskRequest)); getContext().getTasks().put(taskAttemptId, task); @@ -269,10 +267,11 @@ public void run() { } catch (Throwable t) { LOG.error(t.getMessage(), t); fatalError(qmClientService, taskAttemptId, t.getMessage()); + } finally { if(task != null) { task.cleanup(); } - } finally { + callFuture = null; taskRequest = null; } diff --git a/tajo-core/src/main/proto/TajoWorkerProtocol.proto b/tajo-core/src/main/proto/TajoWorkerProtocol.proto index 2324596b1b..715b1e681a 100644 --- a/tajo-core/src/main/proto/TajoWorkerProtocol.proto +++ b/tajo-core/src/main/proto/TajoWorkerProtocol.proto @@ -208,6 +208,7 @@ message TaskAllocationRequestProto { message BatchAllocationRequestProto { required ExecutionBlockIdProto executionBlockId = 1; repeated TaskAllocationRequestProto taskRequest = 2; + optional RunExecutionBlockRequestProto executionBlockRequest = 3; //TODO should be refactored } message BatchAllocationResponseProto { diff --git a/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java index b1a27fa996..f9e37c2558 100644 --- a/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java @@ -33,7 +33,6 @@ import org.apache.tajo.engine.planner.global.MasterPlan; import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.engine.query.TaskRequestImpl; -import org.apache.tajo.ipc.ClientProtos; import org.apache.tajo.master.event.*; import org.apache.tajo.plan.LogicalOptimizer; import org.apache.tajo.plan.LogicalPlan; @@ -42,7 +41,7 @@ import org.apache.tajo.session.Session; import org.apache.tajo.util.CommonTestingUtil; import org.apache.tajo.worker.ExecutionBlockContext; -import org.apache.tajo.worker.Task; +import org.apache.tajo.worker.LegacyTaskImpl; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; @@ -241,15 +240,15 @@ public void testKillTask() throws Throwable { ExecutionBlockContext context = new ExecutionBlockContext(conf, null, null, new QueryContext(conf), null, eid, null, null); - org.apache.tajo.worker.Task task = new Task("test", CommonTestingUtil.getTestDir(), attemptId, + org.apache.tajo.worker.Task task = new LegacyTaskImpl("test", CommonTestingUtil.getTestDir(), attemptId, conf, context, taskRequest); task.kill(); - assertEquals(TajoProtos.TaskAttemptState.TA_KILLED, task.getStatus()); + assertEquals(TajoProtos.TaskAttemptState.TA_KILLED, task.getTaskContext().getState()); try { task.run(); - assertEquals(TajoProtos.TaskAttemptState.TA_KILLED, task.getStatus()); + assertEquals(TajoProtos.TaskAttemptState.TA_KILLED, task.getTaskContext().getState()); } catch (Exception e) { - assertEquals(TajoProtos.TaskAttemptState.TA_KILLED, task.getStatus()); + assertEquals(TajoProtos.TaskAttemptState.TA_KILLED, task.getTaskContext().getState()); } } diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestFetcher.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestFetcher.java index 513eb692b7..65627c1767 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestFetcher.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestFetcher.java @@ -104,13 +104,13 @@ public void testGet() throws IOException { @Test public void testAdjustFetchProcess() { - assertEquals(0.0f, Task.adjustFetchProcess(0, 0), 0); - assertEquals(0.0f, Task.adjustFetchProcess(10, 10), 0); - assertEquals(0.05f, Task.adjustFetchProcess(10, 9), 0); - assertEquals(0.1f, Task.adjustFetchProcess(10, 8), 0); - assertEquals(0.25f, Task.adjustFetchProcess(10, 5), 0); - assertEquals(0.45f, Task.adjustFetchProcess(10, 1), 0); - assertEquals(0.5f, Task.adjustFetchProcess(10, 0), 0); + assertEquals(0.0f, LegacyTaskImpl.adjustFetchProcess(0, 0), 0); + assertEquals(0.0f, LegacyTaskImpl.adjustFetchProcess(10, 10), 0); + assertEquals(0.05f, LegacyTaskImpl.adjustFetchProcess(10, 9), 0); + assertEquals(0.1f, LegacyTaskImpl.adjustFetchProcess(10, 8), 0); + assertEquals(0.25f, LegacyTaskImpl.adjustFetchProcess(10, 5), 0); + assertEquals(0.45f, LegacyTaskImpl.adjustFetchProcess(10, 1), 0); + assertEquals(0.5f, LegacyTaskImpl.adjustFetchProcess(10, 0), 0); } @Test diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java index 7407acc1f9..49e9fa8207 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java @@ -64,7 +64,12 @@ public void setup() { dispatcher.init(conf); dispatcher.start(); - resourceManager = new NodeResourceManager(dispatcher); + AsyncDispatcher taskDispatcher = new AsyncDispatcher(); + TaskManager taskManager = new TaskManager(taskDispatcher, null, dispatcher.getEventHandler()); + taskManager.init(conf); + taskManager.start(); + + resourceManager = new NodeResourceManager(dispatcher, taskManager); resourceManager.init(conf); resourceManager.start(); @@ -142,10 +147,11 @@ public void testNodeResourceDeallocateEvent() throws Exception { assertEquals(requestSize, resourceManager.getAllocatedSize()); //deallocate - for(TaskAllocationRequestProto allocationRequestProto : requestProto.getTaskRequestList()) { - // direct invoke handler for testing - resourceManager.handle(new NodeResourceDeallocateEvent(allocationRequestProto.getResource())); - } +// for(TaskAllocationRequestProto allocationRequestProto : requestProto.getTaskRequestList()) { +// // direct invoke handler for testing +// resourceManager.handle(new NodeResourceDeallocateEvent(allocationRequestProto.getResource())); +// } + Thread.sleep(1000); assertEquals(0, resourceManager.getAllocatedSize()); assertEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); } @@ -187,7 +193,7 @@ public void run() { totalCanceled.addAndGet(proto.getCancellationTaskCount()); } else { complete++; - dispatcher.getEventHandler().handle(new NodeResourceDeallocateEvent(task.getResource())); + // dispatcher.getEventHandler().handle(new NodeResourceDeallocateEvent(task.getResource())); } } catch (Exception e) { fail(e.getMessage()); @@ -216,7 +222,6 @@ protected static Queue createTaskRequests(int memory ExecutionBlockId nullStage = QueryIdFactory.newExecutionBlockId(QueryIdFactory.NULL_QUERY_ID, 0); TaskAttemptId taskAttemptId = QueryIdFactory.newTaskAttemptId(QueryIdFactory.newTaskId(nullStage, i), 0); - TajoWorkerProtocol.TaskRequestProto.Builder builder = TajoWorkerProtocol.TaskRequestProto.newBuilder(); builder.setId(taskAttemptId.getProto()); diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java index fb3c77e27c..134aef81e7 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java @@ -48,7 +48,7 @@ public void setup() { dispatcher.init(conf); dispatcher.start(); - resourceManager = new NodeResourceManager(dispatcher); + resourceManager = new NodeResourceManager(dispatcher, null); resourceManager.init(conf); resourceManager.start(); } From 796dedeb6f9cdda87d7fd3ca8c211423d5cb29e4 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Fri, 29 May 2015 00:26:24 +0900 Subject: [PATCH 16/21] TAJO-1615 --- .../tajo/worker/ExecutionBlockContext.java | 46 +- .../apache/tajo/worker/LegacyTaskImpl.java | 1 + .../tajo/worker/NodeResourceManager.java | 38 +- .../apache/tajo/worker/NodeStatusUpdater.java | 17 +- .../org/apache/tajo/worker/TajoWorker.java | 46 +- .../tajo/worker/TajoWorkerManagerService.java | 9 +- .../tajo/worker/TaskAttemptContext.java | 1 - .../org/apache/tajo/worker/TaskContainer.java | 64 +- .../org/apache/tajo/worker/TaskExecutor.java | 217 ++--- .../java/org/apache/tajo/worker/TaskImpl.java | 838 ++++++++++++++++++ .../org/apache/tajo/worker/TaskManager.java | 208 ++--- .../org/apache/tajo/worker/TaskRunner.java | 1 + .../apache/tajo/worker/TaskRunnerHistory.java | 1 + .../apache/tajo/worker/TaskRunnerManager.java | 12 +- .../event/NodeResourceAllocateEvent.java | 2 +- .../event/NodeResourceDeallocateEvent.java | 2 +- ...nagerEvent.java => NodeResourceEvent.java} | 6 +- .../tajo/worker/event/NodeStatusEvent.java | 8 +- .../tajo/worker/event/TaskRunnerEvent.java | 1 + .../worker/event/TaskRunnerStartEvent.java | 44 +- .../worker/event/TaskRunnerStopEvent.java | 1 + .../tajo/querymaster/TestKillQuery.java | 124 ++- .../worker/MockNodeNodeResourceManager.java | 54 ++ .../apache/tajo/worker/MockTaskExecutor.java | 128 +++ .../tajo/worker/TestNodeResourceManager.java | 69 +- .../tajo/worker/TestNodeStatusUpdater.java | 19 +- 26 files changed, 1476 insertions(+), 481 deletions(-) create mode 100644 tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java rename tajo-core/src/main/java/org/apache/tajo/worker/event/{NodeResourceManagerEvent.java => NodeResourceEvent.java} (80%) create mode 100644 tajo-core/src/test/java/org/apache/tajo/worker/MockNodeNodeResourceManager.java create mode 100644 tajo-core/src/test/java/org/apache/tajo/worker/MockTaskExecutor.java diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java b/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java index 187c01fc00..f5e256ce38 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java @@ -30,6 +30,7 @@ import org.apache.tajo.ExecutionBlockId; import org.apache.tajo.TajoProtos; import org.apache.tajo.TaskAttemptId; +import org.apache.tajo.TaskId; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.ipc.QueryMasterProtocol; @@ -45,7 +46,6 @@ import java.net.ConnectException; import java.net.InetSocketAddress; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentMap; @@ -61,10 +61,10 @@ public class ExecutionBlockContext { private static final Log LOG = LogFactory.getLog(ExecutionBlockContext.class); private TaskRunnerManager manager; - public AtomicInteger completedTasksNum = new AtomicInteger(); - public AtomicInteger succeededTasksNum = new AtomicInteger(); - public AtomicInteger killedTasksNum = new AtomicInteger(); - public AtomicInteger failedTasksNum = new AtomicInteger(); + protected AtomicInteger completedTasksNum = new AtomicInteger(); + protected AtomicInteger succeededTasksNum = new AtomicInteger(); + protected AtomicInteger killedTasksNum = new AtomicInteger(); + protected AtomicInteger failedTasksNum = new AtomicInteger(); private FileSystem localFS; // for input files @@ -95,17 +95,18 @@ public class ExecutionBlockContext { // It keeps all of the query unit attempts while a TaskRunner is running. private final ConcurrentMap tasks = Maps.newConcurrentMap(); + @Deprecated private final ConcurrentMap histories = Maps.newConcurrentMap(); - public ExecutionBlockContext(TajoConf conf, TajoWorker.WorkerContext workerContext, - TaskRunnerManager manager, QueryContext queryContext, String plan, - ExecutionBlockId executionBlockId, WorkerConnectionInfo queryMaster, - PlanProto.ShuffleType shuffleType) throws Throwable { + private final Map taskHistories = Maps.newTreeMap(); + + public ExecutionBlockContext(TajoWorker.WorkerContext workerContext, + TaskRunnerManager manager, RunExecutionBlockRequestProto request) throws IOException { this.manager = manager; - this.executionBlockId = executionBlockId; + this.executionBlockId = new ExecutionBlockId(request.getExecutionBlockId()); this.connManager = RpcClientManager.getInstance(); - this.queryMaster = queryMaster; - this.systemConf = conf; + this.queryMaster = new WorkerConnectionInfo(request.getQueryMaster()); + this.systemConf = workerContext.getConf(); this.reporter = new Reporter(); this.defaultFS = TajoConf.getTajoRootDir(systemConf).getFileSystem(systemConf); this.localFS = FileSystem.getLocal(systemConf); @@ -113,11 +114,11 @@ public ExecutionBlockContext(TajoConf conf, TajoWorker.WorkerContext workerConte // Setup QueryEngine according to the query plan // Here, we can setup row-based query engine or columnar query engine. this.queryEngine = new TajoQueryEngine(systemConf); - this.queryContext = queryContext; - this.plan = plan; + this.queryContext = new QueryContext(workerContext.getConf(), request.getQueryContext()); + this.plan = request.getPlanJson(); this.resource = new ExecutionBlockSharedResource(); this.workerContext = workerContext; - this.shuffleType = shuffleType; + this.shuffleType = request.getShuffleType(); } public void init() throws Throwable { @@ -144,7 +145,7 @@ public void init() throws Throwable { try { getStub().killQuery(null, executionBlockId.getQueryId().getProto(), NullCallback.get()); } catch (Throwable t) { - //ignore + LOG.error(t); } throw e; } @@ -194,7 +195,7 @@ public void stop(){ } } tasks.clear(); - + taskHistories.clear(); resource.release(); RpcClientManager.cleanup(client); } @@ -253,18 +254,29 @@ public Task getTask(TaskAttemptId taskAttemptId){ return tasks.get(taskAttemptId); } + @Deprecated public void stopTaskRunner(String id){ manager.stopTaskRunner(id); } + @Deprecated public TaskRunner getTaskRunner(String taskRunnerId){ return manager.getTaskRunner(taskRunnerId); } + @Deprecated public void addTaskHistory(String taskRunnerId, TaskAttemptId quAttemptId, TaskHistory taskHistory) { histories.get(taskRunnerId).addTaskHistory(quAttemptId, taskHistory); } + public void addTaskHistory(TaskId taskId, TaskHistory taskHistory) { + taskHistories.put(taskId, taskHistory); + } + + public Map getTaskHistories() { + return taskHistories; + } + public void fatalError(TaskAttemptId taskAttemptId, String message) { if (message == null) { message = "No error message"; diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/LegacyTaskImpl.java b/tajo-core/src/main/java/org/apache/tajo/worker/LegacyTaskImpl.java index 9fd8b61d36..0721ef162e 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/LegacyTaskImpl.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/LegacyTaskImpl.java @@ -68,6 +68,7 @@ import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto; import static org.apache.tajo.plan.serder.PlanProto.ShuffleType; +@Deprecated public class LegacyTaskImpl implements Task { private static final Log LOG = LogFactory.getLog(LegacyTaskImpl.class); private static final float FETCHER_PROGRESS = 0.5f; diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java b/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java index c71f03ebfa..e763d13b4f 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/NodeResourceManager.java @@ -32,24 +32,21 @@ import org.apache.tajo.util.CommonTestingUtil; import org.apache.tajo.worker.event.*; -import java.util.concurrent.atomic.AtomicInteger; - import static org.apache.tajo.ipc.TajoWorkerProtocol.*; -public class NodeResourceManager extends AbstractService implements EventHandler { +public class NodeResourceManager extends AbstractService implements EventHandler { private static final Log LOG = LogFactory.getLog(NodeResourceManager.class); private final Dispatcher dispatcher; - private final TaskManager taskManager; + private final EventHandler taskEventHandler; private NodeResource totalResource; private NodeResource availableResource; - private AtomicInteger allocatedSize; private TajoConf tajoConf; - public NodeResourceManager(Dispatcher dispatcher, TaskManager taskManager) { + public NodeResourceManager(Dispatcher dispatcher, EventHandler taskEventHandler) { super(NodeResourceManager.class.getName()); this.dispatcher = dispatcher; - this.taskManager = taskManager; + this.taskEventHandler = taskEventHandler; } @Override @@ -60,14 +57,14 @@ protected void serviceInit(Configuration conf) throws Exception { this.tajoConf = (TajoConf)conf; this.totalResource = createWorkerResource(tajoConf); this.availableResource = NodeResources.clone(totalResource); - this.dispatcher.register(NodeResourceManagerEvent.EventType.class, this); - this.allocatedSize = new AtomicInteger(); + this.dispatcher.register(NodeResourceEvent.EventType.class, this); + super.serviceInit(conf); LOG.info("Initialized NodeResourceManager for " + totalResource); } @Override - public void handle(NodeResourceManagerEvent event) { + public void handle(NodeResourceEvent event) { if (event instanceof NodeResourceAllocateEvent) { NodeResourceAllocateEvent allocateEvent = (NodeResourceAllocateEvent) event; @@ -75,15 +72,13 @@ public void handle(NodeResourceManagerEvent event) { for (TaskAllocationRequestProto request : allocateEvent.getRequest().getTaskRequestList()) { NodeResource resource = new NodeResource(request.getResource()); if (allocate(resource)) { - allocatedSize.incrementAndGet(); if(allocateEvent.getRequest().hasExecutionBlockRequest()){ //send ExecutionBlock start event to TaskManager - taskManager.getEventHandler().handle( - new ExecutionBlockStartEvent(allocateEvent.getRequest().getExecutionBlockRequest())); + startExecutionBlock(allocateEvent.getRequest().getExecutionBlockRequest()); } //send task start event to TaskExecutor - taskManager.getEventHandler().handle(new TaskStartEvent(request.getTaskRequest(), resource)); + startTask(request.getTaskRequest(), resource); } else { // reject the exceeded requests response.addCancellationTask(request); @@ -92,13 +87,12 @@ public void handle(NodeResourceManagerEvent event) { allocateEvent.getCallback().run(response.build()); } else if (event instanceof NodeResourceDeallocateEvent) { - allocatedSize.decrementAndGet(); NodeResourceDeallocateEvent deallocateEvent = (NodeResourceDeallocateEvent) event; release(deallocateEvent.getResource()); // send current resource to ResourceTracker getDispatcher().getEventHandler().handle( - new NodeStatusEvent(NodeStatusEvent.EventType.REPORT_RESOURCE, getAvailableResource())); + new NodeStatusEvent(NodeStatusEvent.EventType.REPORT_RESOURCE)); } } @@ -114,10 +108,6 @@ protected NodeResource getAvailableResource() { return availableResource; } - public int getAllocatedSize() { - return allocatedSize.get(); - } - private boolean allocate(NodeResource resource) { //TODO consider the jvm free memory if (NodeResources.fitsIn(resource, availableResource)) { @@ -127,6 +117,14 @@ private boolean allocate(NodeResource resource) { return false; } + protected void startExecutionBlock(RunExecutionBlockRequestProto request) { + taskEventHandler.handle(new ExecutionBlockStartEvent(request)); + } + + protected void startTask(TaskRequestProto request, NodeResource resource) { + taskEventHandler.handle(new TaskStartEvent(request, resource)); + } + private void release(NodeResource resource) { NodeResources.addTo(availableResource, resource); } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java b/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java index 84ac419066..6bf4a67879 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java @@ -107,14 +107,7 @@ public void serviceStop() throws Exception { @Override public void handle(NodeStatusEvent event) { - switch (event.getType()) { - case REPORT_RESOURCE: - heartBeatRequestQueue.add(event); //batch report to ResourceTracker - break; - case FLUSH_REPORTS: - heartBeatRequestQueue.add(event); //flush report to ResourceTracker - break; - } + heartBeatRequestQueue.add(event); } public int getQueueSize() { @@ -231,8 +224,8 @@ public void run() { } if (!events.isEmpty()) { - // send last available resource; - lastResponse = sendHeartbeat(createResourceReport(events.get(events.size() - 1).getResource())); + // send current available resource; + lastResponse = sendHeartbeat(createResourceReport(nodeResourceManager.getAvailableResource())); } else { // send ping; lastResponse = sendHeartbeat(createHeartBeatReport()); @@ -250,10 +243,10 @@ public void run() { } } catch (NoSuchMethodException nsme) { LOG.fatal(nsme.getMessage(), nsme); - Runtime.getRuntime().halt(1); + Runtime.getRuntime().halt(-1); } catch (ClassNotFoundException cnfe) { LOG.fatal(cnfe.getMessage(), cnfe); - Runtime.getRuntime().halt(1); + Runtime.getRuntime().halt(-1); } catch (Exception e) { LOG.error(e.getMessage(), e); if (!isStopped) { diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java index 6f4e89919a..fbd070e5d0 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java @@ -163,7 +163,7 @@ public void serviceInit(Configuration conf) throws Exception { serviceTracker = ServiceTrackerFactory.get(systemConf); - this.workerContext = new WorkerContext(); + this.workerContext = new TajoWorkerContext(); this.lDirAllocator = new LocalDirAllocator(ConfVars.WORKER_TEMPORAL_DIR.varname); String resourceManagerClassName = systemConf.getVar(ConfVars.RESOURCE_MANAGER_CLASS); @@ -386,7 +386,45 @@ public void serviceStop() throws Exception { LOG.info("TajoWorker main thread exiting"); } - public class WorkerContext { + public interface WorkerContext { + QueryMaster getQueryMaster(); + + TajoConf getConf(); + + ServiceTracker getServiceTracker(); + + QueryMasterManagerService getQueryMasterManagerService(); + + TaskRunnerManager getTaskRunnerManager(); + + CatalogService getCatalog(); + + WorkerConnectionInfo getConnectionInfo(); + + String getWorkerName(); + + LocalDirAllocator getLocalDirAllocator(); + + ClusterResourceSummary getClusterResource(); + + TajoSystemMetrics getWorkerSystemMetrics(); + + HashShuffleAppenderManager getHashShuffleAppenderManager(); + + HistoryWriter getTaskHistoryWriter(); + + HistoryReader getHistoryReader(); + + void cleanup(String strPath); + + void cleanupTemporalDirectories(); + + void setClusterResource(ClusterResourceSummary clusterResource); + + void setNumClusterNodes(int numClusterNodes); + } + + class TajoWorkerContext implements WorkerContext { public QueryMaster getQueryMaster() { if (queryMasterManagerService == null) { return null; @@ -430,7 +468,7 @@ public LocalDirAllocator getLocalDirAllocator(){ return lDirAllocator; } - protected void cleanup(String strPath) { + public void cleanup(String strPath) { if (deletionService == null) return; LocalDirAllocator lDirAllocator = new LocalDirAllocator(ConfVars.WORKER_TEMPORAL_DIR.varname); @@ -446,7 +484,7 @@ protected void cleanup(String strPath) { } } - protected void cleanupTemporalDirectories() { + public void cleanupTemporalDirectories() { if (deletionService == null) return; LocalDirAllocator lDirAllocator = new LocalDirAllocator(ConfVars.WORKER_TEMPORAL_DIR.varname); diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java index bbf8564566..de8afe8377 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java @@ -116,14 +116,7 @@ public void startExecutionBlock(RpcController controller, workerContext.getWorkerSystemMetrics().counter("query", "executedExecutionBlocksNum").inc(); try { - workerContext.getTaskRunnerManager().getEventHandler().handle(new TaskRunnerStartEvent( - new WorkerConnectionInfo(request.getQueryMaster()) - , new ExecutionBlockId(request.getExecutionBlockId()) - , request.getContainerId() - , new QueryContext(workerContext.getConf(), request.getQueryContext()), - request.getPlanJson(), - request.getShuffleType() - )); + workerContext.getTaskRunnerManager().getEventHandler().handle(new TaskRunnerStartEvent(request)); done.run(TajoWorker.TRUE_PROTO); } catch (Throwable t) { LOG.error(t.getMessage(), t); diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java index de0bd9b19b..762e087488 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java @@ -46,7 +46,6 @@ import java.util.*; import java.util.Map.Entry; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.atomic.AtomicBoolean; import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto; diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskContainer.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskContainer.java index fd083d1c3c..1d2daa6c1d 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskContainer.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskContainer.java @@ -18,28 +18,9 @@ package org.apache.tajo.worker; -import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.service.AbstractService; -import org.apache.tajo.ExecutionBlockId; -import org.apache.tajo.TaskAttemptId; -import org.apache.tajo.conf.TajoConf; -import org.apache.tajo.conf.TajoConf.ConfVars; -import org.apache.tajo.engine.query.TaskRequestImpl; -import org.apache.tajo.ipc.QueryMasterProtocol.QueryMasterProtocolService; -import org.apache.tajo.master.container.TajoContainerId; -import org.apache.tajo.master.container.TajoContainerIdPBImpl; -import org.apache.tajo.master.container.TajoConverterUtils; -import org.apache.tajo.rpc.CallFuture; -import org.apache.tajo.rpc.NullCallback; - -import java.io.IOException; -import java.util.concurrent.*; - -import static org.apache.tajo.ipc.TajoWorkerProtocol.*; +import org.apache.tajo.TajoProtos; /** * The driver class for Tajo Task processing. @@ -47,7 +28,6 @@ public class TaskContainer implements Runnable { private static final Log LOG = LogFactory.getLog(TaskContainer.class); - // Contains the object references related for TaskRunner private final TaskExecutor executor; private final int sequenceId; @@ -56,26 +36,29 @@ public TaskContainer(int sequenceId, TaskExecutor executor) { this.executor = executor; } - public void init() throws IOException { - //if (executionBlockContext.isStopped()) return; - -// LOG.info("Initializing: " + task.getId()); -// getContext().getWorkerContext().getWorkerSystemMetrics().counter("query", "task").inc(); - } - @Override public void run() { - while (true) { + while (!executor.isStopped()) { + Task task = null; try { task = executor.getNextTask(); - LOG.debug(sequenceId + " got task:" + task.getTaskContext().getTaskId()); - TaskAttemptContext taskAttemptContext = task.getTaskContext(); - task.init(); + if (task.getExecutionBlockContext() != null) { + task.getExecutionBlockContext().getWorkerContext().getWorkerSystemMetrics().counter("query", "task").inc(); + } + + if (LOG.isDebugEnabled()) { + LOG.debug(sequenceId + TaskContainer.class.getSimpleName() + + " got task:" + task.getTaskContext().getTaskId()); + } + + TaskAttemptContext taskAttemptContext = task.getTaskContext(); if (taskAttemptContext.isStopped()) return; + task.init(); + if (task.hasFetchPhase()) { task.fetch(); // The fetch is performed in an asynchronous way. } @@ -85,13 +68,20 @@ public void run() { } task.cleanup(); - } catch (Exception t) { - LOG.error(t.getMessage(), t); - if(task != null){ - task.getExecutionBlockContext().fatalError(task.getTaskContext().getTaskId(), t.getMessage()); + } catch (Exception e) { + LOG.error(e.getMessage(), e); + if (task != null) { + try { + task.abort(); + task.getExecutionBlockContext().fatalError(task.getTaskContext().getTaskId(), e.getMessage()); + } catch (Throwable t) { + LOG.fatal(t.getMessage(), t); + } } } finally { - executor.stopTask(task); + if (task != null) { + executor.stopTask(task.getTaskContext().getTaskId()); + } } } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java index 075fadac60..9235404d6d 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java @@ -32,6 +32,7 @@ import org.apache.tajo.conf.TajoConf; import org.apache.tajo.conf.TajoConf.ConfVars; import org.apache.tajo.engine.query.TaskRequest; +import org.apache.tajo.engine.query.TaskRequestImpl; import org.apache.tajo.ipc.TajoWorkerProtocol; import org.apache.tajo.resource.NodeResource; import org.apache.tajo.resource.NodeResources; @@ -42,8 +43,7 @@ import java.util.concurrent.*; import java.util.concurrent.atomic.AtomicInteger; -public class TaskExecutor extends AbstractService - implements EventHandler { +public class TaskExecutor extends AbstractService implements EventHandler { private static final Log LOG = LogFactory.getLog(TaskExecutor.class); private final TaskManager taskManager; @@ -54,6 +54,7 @@ public class TaskExecutor extends AbstractService private ThreadPoolExecutor fetcherExecutor; private ExecutorService threadPool; private TajoConf tajoConf; + private volatile boolean isStopped; public TaskExecutor(TaskManager taskManager, EventHandler rmEventHandler) { super(TaskExecutor.class.getName()); @@ -69,7 +70,9 @@ protected void serviceInit(Configuration conf) throws Exception { if (!(conf instanceof TajoConf)) { throw new IllegalArgumentException("Configuration must be a TajoConf instance"); } + this.tajoConf = (TajoConf) conf; + this.taskManager.getDispatcher().register(TaskExecutorEvent.EventType.class, this); super.serviceInit(conf); } @@ -94,6 +97,23 @@ protected void serviceStart() throws Exception { LOG.info("Started TaskExecutor[" + nThreads + "], Fetcher executor[" + maxFetcherThreads + "]"); } + @Override + protected void serviceStop() throws Exception { + isStopped = true; + + threadPool.shutdown(); + fetcherExecutor.shutdown(); + super.serviceStop(); + } + + public boolean isStopped() { + return isStopped; + } + + public int getRunningTasks() { + return runningTasks.get(); + } + /** * This will block until a task is available. */ @@ -107,169 +127,62 @@ protected Task getNextTask() { return task; } - - protected void beforeExecute(Thread t, Runnable r) { - System.out.println(r); - Future future = (Future) r; - - TaskContainer container = null; -// try { -// container = future.get(); -// } catch (InterruptedException e) { -// e.printStackTrace(); -// } catch (ExecutionException e) { -// e.printStackTrace(); -// } -// try { -// container.init(); -// } catch (IOException e) { -// LOG.error(e.getMessage(), e); -// container.getTask().abort(); -// container.getContext().fatalError(container.getTask().getId(), e.getMessage()); - //} - } - - - protected void afterExecute(Runnable r, Throwable t) { - TaskContainer container = null; - try { - container = ((Future) r).get(); - } catch (InterruptedException e) { - e.printStackTrace(); - } catch (ExecutionException e) { - e.printStackTrace(); - } - try { - - if (t != null) { - LOG.error(t.getMessage(), t); - // container.getContext().fatalError(container.getTask().getId(), t.getMessage()); - return; - } - - try { - // container.stop(); - } catch (Throwable throwable) { - LOG.error(throwable.getMessage(), throwable); - //container.getContext().fatalError(container.getTask().getId(), throwable.getMessage()); - } - } finally { - - } - } - @SuppressWarnings("unchecked") - protected void stopTask(Task task) { - //FIXME change to TaskStopEvent ? - if(task != null) { - rmEventHandler - .handle(new NodeResourceDeallocateEvent(allocatedResourceMap.remove(task.getTaskContext().getTaskId()))); - runningTasks.decrementAndGet(); - } - + protected void stopTask(TaskAttemptId taskId) { + runningTasks.decrementAndGet(); + rmEventHandler.handle(new NodeResourceDeallocateEvent(allocatedResourceMap.remove(taskId))); } + protected ExecutorService getFetcherExecutor() { return fetcherExecutor; } - /*FIXME*/ - protected Task createTask(final ExecutionBlockContext context, TajoWorkerProtocol.TaskRequestProto taskRequest){ - - final TaskAttemptId taskAttemptId = new TaskAttemptId(taskRequest.getId()); -// if (getContext().getTasks().containsKey(taskAttemptId)) { -// LOG.error("Duplicate Task Attempt: " + taskAttemptId); -// fatalError(qmClientService, taskAttemptId, "Duplicate Task Attempt: " + taskAttemptId); -// continue; -// } - final TaskAttemptContext taskAttemptContext = new TaskAttemptContext(null,context,taskAttemptId, null, null); - //LOG.info("Initializing: " + taskAttemptId); - return new Task() { - @Override - public void init() throws IOException { - - } - - @Override - public void fetch() { - } - - @Override - public void run() throws Exception { - - } - - @Override - public void kill() { - - } - - @Override - public void abort() { - - } - - @Override - public void cleanup() { - - } - - @Override - public boolean hasFetchPhase() { - return false; - } - - @Override - public boolean isProgressChanged() { - return false; - } - - @Override - public boolean isStopped() { - return taskAttemptContext.isStopped(); - } - - @Override - public void updateProgress() { - - } - - @Override - public TaskAttemptContext getTaskContext() { - return taskAttemptContext; - } - - @Override - public ExecutionBlockContext getExecutionBlockContext() { - return context; - } - - @Override - public TajoWorkerProtocol.TaskStatusProto getReport() { - return null; - } - }; + protected Task createTask(ExecutionBlockContext executionBlockContext, + TajoWorkerProtocol.TaskRequestProto taskRequest) throws IOException { + Task task = null; + TaskAttemptId taskAttemptId = new TaskAttemptId(taskRequest.getId()); + if (executionBlockContext.getTasks().containsKey(taskAttemptId)) { + String errorMessage = "Duplicate Task Attempt: " + taskAttemptId; + LOG.error(errorMessage); + executionBlockContext.fatalError(taskAttemptId, errorMessage); + } else { + task = new TaskImpl(new TaskRequestImpl(taskRequest), executionBlockContext, getFetcherExecutor()); + executionBlockContext.getTasks().put(task.getTaskContext().getTaskId(), task); + } + return task; } @Override public void handle(TaskExecutorEvent event) { - if(event instanceof TaskStartEvent) - switch (event.getType()) { - case START:{ - final TaskStartEvent startEvent = (TaskStartEvent)event; - allocatedResourceMap.put(startEvent.getTaskId(), startEvent.getAllocatedResource()); + if (event instanceof TaskStartEvent) { + TaskStartEvent startEvent = (TaskStartEvent) event; + allocatedResourceMap.put(startEvent.getTaskId(), startEvent.getAllocatedResource()); - ExecutionBlockContext context = taskManager.getExecutionBlockContext( - startEvent.getTaskId().getTaskId().getExecutionBlockId()); + ExecutionBlockContext context = taskManager.getExecutionBlockContext( + startEvent.getTaskId().getTaskId().getExecutionBlockId()); - try { - taskQueue.put(createTask(context, startEvent.getTaskRequest())); - runningTasks.incrementAndGet(); - } catch (InterruptedException e) { - LOG.fatal(e.getMessage(), e); - } - } - } + try { + Task task = createTask(context, startEvent.getTaskRequest()); + if (task != null) { + if (LOG.isDebugEnabled()) { + LOG.debug("Arrival task: " + task.getTaskContext().getTaskId() + + ", allocated resource: " + startEvent.getAllocatedResource()); + } + taskQueue.put(task); + runningTasks.incrementAndGet(); + } else { + LOG.warn("Release duplicate task resource: " + startEvent.getAllocatedResource()); + stopTask(startEvent.getTaskId()); + } + } catch (InterruptedException e) { + if (!isStopped) { + LOG.fatal(e.getMessage(), e); + } + } catch (IOException e) { + stopTask(startEvent.getTaskId()); + } + } } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java new file mode 100644 index 0000000000..be3960b0ca --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskImpl.java @@ -0,0 +1,838 @@ +/** + * 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.tajo.worker; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import io.netty.handler.codec.http.QueryStringDecoder; +import org.apache.commons.lang.exception.ExceptionUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.tajo.TajoProtos; +import org.apache.tajo.TajoProtos.TaskAttemptState; +import org.apache.tajo.TaskAttemptId; +import org.apache.tajo.catalog.Schema; +import org.apache.tajo.catalog.TableDesc; +import org.apache.tajo.catalog.TableMeta; +import org.apache.tajo.catalog.proto.CatalogProtos; +import org.apache.tajo.catalog.statistics.TableStats; +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.engine.planner.physical.PhysicalExec; +import org.apache.tajo.engine.query.QueryContext; +import org.apache.tajo.engine.query.TaskRequest; +import org.apache.tajo.ipc.QueryMasterProtocol; +import org.apache.tajo.ipc.TajoWorkerProtocol.*; +import org.apache.tajo.ipc.TajoWorkerProtocol.EnforceProperty.EnforceType; +import org.apache.tajo.master.cluster.WorkerConnectionInfo; +import org.apache.tajo.plan.function.python.TajoScriptEngine; +import org.apache.tajo.plan.logical.*; +import org.apache.tajo.plan.serder.LogicalNodeDeserializer; +import org.apache.tajo.plan.util.PlannerUtil; +import org.apache.tajo.pullserver.TajoPullServerService; +import org.apache.tajo.pullserver.retriever.FileChunk; +import org.apache.tajo.rpc.NullCallback; +import org.apache.tajo.storage.*; +import org.apache.tajo.storage.fragment.FileFragment; +import org.apache.tajo.util.NetUtils; + +import java.io.File; +import java.io.IOException; +import java.net.InetAddress; +import java.net.URI; +import java.util.*; +import java.util.Map.Entry; +import java.util.concurrent.ExecutorService; + +import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto; +import static org.apache.tajo.plan.serder.PlanProto.ShuffleType; + +public class TaskImpl implements Task { + private static final Log LOG = LogFactory.getLog(TaskImpl.class); + private static final float FETCHER_PROGRESS = 0.5f; + + private final TajoConf systemConf; + private final QueryContext queryContext; + private final ExecutionBlockContext executionBlockContext; + private final TaskRequest request; + private final Map descs; + private final TableStats inputStats; + private final ExecutorService fetcherExecutor; + private final Path taskDir; + + private final TaskAttemptContext context; + private List fetcherRunners; + private LogicalNode plan; + private PhysicalExec executor; + + private boolean interQuery; + private Path inputTableBaseDir; + + private long startTime; + private long endTime; + + private List localChunks; + // TODO - to be refactored + private ShuffleType shuffleType = null; + private Schema finalSchema = null; + + private TupleComparator sortComp = null; + + public TaskImpl(final TaskRequest request, + final ExecutionBlockContext executionBlockContext, + final ExecutorService fetcherExecutor) throws IOException { + + this.request = request; + this.executionBlockContext = executionBlockContext; + this.systemConf = executionBlockContext.getConf(); + this.queryContext = request.getQueryContext(systemConf); + this.inputStats = new TableStats(); + this.fetcherRunners = Lists.newArrayList(); + this.fetcherExecutor = fetcherExecutor; + this.descs = Maps.newHashMap(); + + Path baseDirPath = executionBlockContext.createBaseDir(); + LOG.info("Task basedir is created (" + baseDirPath +")"); + TaskAttemptId taskAttemptId = request.getId(); + + this.taskDir = StorageUtil.concatPath(baseDirPath, + taskAttemptId.getTaskId().getId() + "_" + taskAttemptId.getId()); + this.context = new TaskAttemptContext(queryContext, executionBlockContext, taskAttemptId, + request.getFragments().toArray(new FragmentProto[request.getFragments().size()]), taskDir); + this.context.setDataChannel(request.getDataChannel()); + this.context.setEnforcer(request.getEnforcer()); + this.context.setState(TaskAttemptState.TA_PENDING); + } + + public void initPlan() throws IOException { + plan = LogicalNodeDeserializer.deserialize(queryContext, context.getEvalContext(), request.getPlan()); + LogicalNode [] scanNode = PlannerUtil.findAllNodes(plan, NodeType.SCAN); + if (scanNode != null) { + for (LogicalNode node : scanNode) { + ScanNode scan = (ScanNode) node; + descs.put(scan.getCanonicalName(), scan.getTableDesc()); + } + } + + LogicalNode [] partitionScanNode = PlannerUtil.findAllNodes(plan, NodeType.PARTITIONS_SCAN); + if (partitionScanNode != null) { + for (LogicalNode node : partitionScanNode) { + PartitionedTableScanNode scan = (PartitionedTableScanNode) node; + descs.put(scan.getCanonicalName(), scan.getTableDesc()); + } + } + + interQuery = request.getProto().getInterQuery(); + if (interQuery) { + context.setInterQuery(); + this.shuffleType = context.getDataChannel().getShuffleType(); + + if (shuffleType == ShuffleType.RANGE_SHUFFLE) { + SortNode sortNode = PlannerUtil.findTopNode(plan, NodeType.SORT); + this.finalSchema = PlannerUtil.sortSpecsToSchema(sortNode.getSortKeys()); + this.sortComp = new BaseTupleComparator(finalSchema, sortNode.getSortKeys()); + } + } else { + Path outFilePath = ((FileTablespace) TableSpaceManager.getFileStorageManager(systemConf)) + .getAppenderFilePath(getId(), queryContext.getStagingDir()); + LOG.info("Output File Path: " + outFilePath); + context.setOutputPath(outFilePath); + } + + this.localChunks = Collections.synchronizedList(new ArrayList()); + LOG.info("=================================="); + LOG.info("* Stage " + request.getId() + " is initialized"); + LOG.info("* InterQuery: " + interQuery + + (interQuery ? ", Use " + this.shuffleType + " shuffle" : "") + + ", Fragments (num: " + request.getFragments().size() + ")" + + ", Fetches (total:" + request.getFetches().size() + ") :"); + + if(LOG.isDebugEnabled()) { + for (FetchImpl f : request.getFetches()) { + LOG.debug("Table Id: " + f.getName() + ", Simple URIs: " + f.getSimpleURIs()); + } + } + LOG.info("* Local task dir: " + taskDir); + if(LOG.isDebugEnabled()) { + LOG.debug("* plan:\n"); + LOG.debug(plan.toString()); + } + LOG.info("=================================="); + } + + private void startScriptExecutors() throws IOException { + for (TajoScriptEngine executor : context.getEvalContext().getAllScriptEngines()) { + executor.start(systemConf); + } + } + + private void stopScriptExecutors() { + for (TajoScriptEngine executor : context.getEvalContext().getAllScriptEngines()) { + executor.shutdown(); + } + } + + @Override + public void init() throws IOException { + LOG.info("Initializing: " + getId()); + + initPlan(); + startScriptExecutors(); + + if (context.getState() == TaskAttemptState.TA_PENDING) { + // initialize a task temporal dir + FileSystem localFS = executionBlockContext.getLocalFS(); + localFS.mkdirs(taskDir); + + if (request.getFetches().size() > 0) { + inputTableBaseDir = localFS.makeQualified( + executionBlockContext.getLocalDirAllocator().getLocalPathForWrite( + getTaskAttemptDir(context.getTaskId()).toString(), systemConf)); + localFS.mkdirs(inputTableBaseDir); + Path tableDir; + for (String inputTable : context.getInputTables()) { + tableDir = new Path(inputTableBaseDir, inputTable); + if (!localFS.exists(tableDir)) { + LOG.info("the directory is created " + tableDir.toUri()); + localFS.mkdirs(tableDir); + } + } + } + // for localizing the intermediate data + fetcherRunners.addAll(getFetchRunners(context, request.getFetches())); + } + } + + private TaskAttemptId getId() { + return context.getTaskId(); + } + + public String toString() { + return "TaskId: " + this.getId() + " Status: " + context.getState(); + } + + @Override + public boolean isStopped() { + return context.isStopped(); + } + + @Override + public TaskAttemptContext getTaskContext() { + return context; + } + + @Override + public ExecutionBlockContext getExecutionBlockContext() { + return executionBlockContext; + } + + @Override + public boolean hasFetchPhase() { + return fetcherRunners.size() > 0; + } + + @Override + public void fetch() { + for (Fetcher f : fetcherRunners) { + fetcherExecutor.submit(new FetchRunner(context, f)); + } + } + + @Override + public void kill() { + stopScriptExecutors(); + context.setState(TaskAttemptState.TA_KILLED); + context.stop(); + } + + @Override + public void abort() { + stopScriptExecutors(); + context.setState(TaskAttemptState.TA_FAILED); + context.stop(); + } + + @Override + public TaskStatusProto getReport() { + TaskStatusProto.Builder builder = TaskStatusProto.newBuilder(); + builder.setWorkerName(executionBlockContext.getWorkerContext().getConnectionInfo().getHostAndPeerRpcPort()); + builder.setId(context.getTaskId().getProto()) + .setProgress(context.getProgress()) + .setState(context.getState()); + + builder.setInputStats(reloadInputStats()); + + if (context.getResultStats() != null) { + builder.setResultStats(context.getResultStats().getProto()); + } + return builder.build(); + } + + @Override + public boolean isProgressChanged() { + return context.isProgressChanged(); + } + + @Override + public void updateProgress() { + if(context != null && context.isStopped()){ + return; + } + + if (executor != null && context.getProgress() < 1.0f) { + context.setExecutorProgress(executor.getProgress()); + } + } + + private CatalogProtos.TableStatsProto reloadInputStats() { + synchronized(inputStats) { + if (this.executor == null) { + return inputStats.getProto(); + } + + TableStats executorInputStats = this.executor.getInputStats(); + + if (executorInputStats != null) { + inputStats.setValues(executorInputStats); + } + return inputStats.getProto(); + } + } + + private TaskCompletionReport getTaskCompletionReport() { + TaskCompletionReport.Builder builder = TaskCompletionReport.newBuilder(); + builder.setId(context.getTaskId().getProto()); + + builder.setInputStats(reloadInputStats()); + + if (context.hasResultStats()) { + builder.setResultStats(context.getResultStats().getProto()); + } else { + builder.setResultStats(new TableStats().getProto()); + } + + Iterator> it = context.getShuffleFileOutputs(); + if (it.hasNext()) { + do { + Entry entry = it.next(); + ShuffleFileOutput.Builder part = ShuffleFileOutput.newBuilder(); + part.setPartId(entry.getKey()); + + // Set output volume + if (context.getPartitionOutputVolume() != null) { + for (Entry e : context.getPartitionOutputVolume().entrySet()) { + if (entry.getKey().equals(e.getKey())) { + part.setVolume(e.getValue().longValue()); + break; + } + } + } + + builder.addShuffleFileOutputs(part.build()); + } while (it.hasNext()); + } + + return builder.build(); + } + + private void waitForFetch() throws InterruptedException, IOException { + context.getFetchLatch().await(); + LOG.info(context.getTaskId() + " All fetches are done!"); + Collection inputs = Lists.newArrayList(context.getInputTables()); + + // Get all broadcasted tables + Set broadcastTableNames = new HashSet(); + List broadcasts = context.getEnforcer().getEnforceProperties(EnforceType.BROADCAST); + if (broadcasts != null) { + for (EnforceProperty eachBroadcast : broadcasts) { + broadcastTableNames.add(eachBroadcast.getBroadcast().getTableName()); + } + } + + // localize the fetched data and skip the broadcast table + for (String inputTable: inputs) { + if (broadcastTableNames.contains(inputTable)) { + continue; + } + File tableDir = new File(context.getFetchIn(), inputTable); + FileFragment[] frags = localizeFetchedData(tableDir, inputTable, descs.get(inputTable).getMeta()); + context.updateAssignedFragments(inputTable, frags); + } + } + + @Override + public void run() throws Exception { + startTime = System.currentTimeMillis(); + Throwable error = null; + + try { + if(!context.isStopped()) { + context.setState(TajoProtos.TaskAttemptState.TA_RUNNING); + if (context.hasFetchPhase()) { + // If the fetch is still in progress, the query unit must wait for complete. + waitForFetch(); + context.setFetcherProgress(FETCHER_PROGRESS); + updateProgress(); + } + + this.executor = executionBlockContext.getTQueryEngine().createPlan(context, plan); + this.executor.init(); + + while(!context.isStopped() && executor.next() != null) { + } + } + } catch (Throwable e) { + error = e ; + LOG.error(e.getMessage(), e); + stopScriptExecutors(); + context.stop(); + } finally { + if (executor != null) { + try { + executor.close(); + reloadInputStats(); + } catch (IOException e) { + LOG.error(e, e); + } + this.executor = null; + } + + executionBlockContext.completedTasksNum.incrementAndGet(); + context.getHashShuffleAppenderManager().finalizeTask(getId()); + + QueryMasterProtocol.QueryMasterProtocolService.Interface queryMasterStub = executionBlockContext.getStub(); + if (context.isStopped()) { + context.setExecutorProgress(0.0f); + + if (context.getState() == TaskAttemptState.TA_KILLED) { + queryMasterStub.statusUpdate(null, getReport(), NullCallback.get()); + executionBlockContext.killedTasksNum.incrementAndGet(); + } else { + context.setState(TaskAttemptState.TA_FAILED); + TaskFatalErrorReport.Builder errorBuilder = + TaskFatalErrorReport.newBuilder() + .setId(getId().getProto()); + if (error != null) { + if (error.getMessage() == null) { + errorBuilder.setErrorMessage(error.getClass().getCanonicalName()); + } else { + errorBuilder.setErrorMessage(error.getMessage()); + } + errorBuilder.setErrorTrace(ExceptionUtils.getStackTrace(error)); + } + + queryMasterStub.fatalError(null, errorBuilder.build(), NullCallback.get()); + executionBlockContext.failedTasksNum.incrementAndGet(); + } + } else { + // if successful + context.stop(); + context.setProgress(1.0f); + context.setState(TaskAttemptState.TA_SUCCEEDED); + executionBlockContext.succeededTasksNum.incrementAndGet(); + + TaskCompletionReport report = getTaskCompletionReport(); + queryMasterStub.done(null, report, NullCallback.get()); + } + endTime = System.currentTimeMillis(); + LOG.info(context.getTaskId() + " completed. " + + "Worker's task counter - total:" + executionBlockContext.completedTasksNum.intValue() + + ", succeeded: " + executionBlockContext.succeededTasksNum.intValue() + + ", killed: " + executionBlockContext.killedTasksNum.intValue() + + ", failed: " + executionBlockContext.failedTasksNum.intValue()); + } + } + + @Override + public void cleanup() { + TaskHistory taskHistory = createTaskHistory(); + executionBlockContext.addTaskHistory(getId().getTaskId(), taskHistory); + executionBlockContext.getTasks().remove(getId()); + + fetcherRunners.clear(); + fetcherRunners = null; + try { + if(executor != null) { + executor.close(); + executor = null; + } + } catch (IOException e) { + LOG.fatal(e.getMessage(), e); + } + + executionBlockContext.getWorkerContext().getTaskHistoryWriter().appendHistory(taskHistory); + stopScriptExecutors(); + } + + public TaskHistory createTaskHistory() { + TaskHistory taskHistory = null; + try { + taskHistory = new TaskHistory(context.getTaskId(), context.getState(), context.getProgress(), + startTime, endTime, reloadInputStats()); + + if (context.getOutputPath() != null) { + taskHistory.setOutputPath(context.getOutputPath().toString()); + } + + if (context.getWorkDir() != null) { + taskHistory.setWorkingPath(context.getWorkDir().toString()); + } + + if (context.getResultStats() != null) { + taskHistory.setOutputStats(context.getResultStats().getProto()); + } + + if (hasFetchPhase()) { + taskHistory.setTotalFetchCount(fetcherRunners.size()); + int i = 0; + FetcherHistoryProto.Builder builder = FetcherHistoryProto.newBuilder(); + for (Fetcher fetcher : fetcherRunners) { + // TODO store the fetcher histories + if (systemConf.getBoolVar(TajoConf.ConfVars.$DEBUG_ENABLED)) { + builder.setStartTime(fetcher.getStartTime()); + builder.setFinishTime(fetcher.getFinishTime()); + builder.setFileLength(fetcher.getFileLen()); + builder.setMessageReceivedCount(fetcher.getMessageReceiveCount()); + builder.setState(fetcher.getState()); + + taskHistory.addFetcherHistory(builder.build()); + } + if (fetcher.getState() == TajoProtos.FetcherState.FETCH_FINISHED) i++; + } + taskHistory.setFinishedFetchCount(i); + } + } catch (Exception e) { + LOG.warn(e.getMessage(), e); + } + + return taskHistory; + } + + public int hashCode() { + return context.hashCode(); + } + + public boolean equals(Object obj) { + if (obj instanceof TaskImpl) { + TaskImpl other = (TaskImpl) obj; + return this.context.equals(other.context); + } + return false; + } + + private FileFragment[] localizeFetchedData(File file, String name, TableMeta meta) + throws IOException { + Configuration c = new Configuration(systemConf); + c.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, "file:///"); + FileSystem fs = FileSystem.get(c); + Path tablePath = new Path(file.getAbsolutePath()); + + List listTablets = new ArrayList(); + FileFragment tablet; + + FileStatus[] fileLists = fs.listStatus(tablePath); + for (FileStatus f : fileLists) { + if (f.getLen() == 0) { + continue; + } + tablet = new FileFragment(name, f.getPath(), 0l, f.getLen()); + listTablets.add(tablet); + } + + // Special treatment for locally pseudo fetched chunks + synchronized (localChunks) { + for (FileChunk chunk : localChunks) { + if (name.equals(chunk.getEbId())) { + tablet = new FileFragment(name, new Path(chunk.getFile().getPath()), chunk.startOffset(), chunk.length()); + listTablets.add(tablet); + LOG.info("One local chunk is added to listTablets"); + } + } + } + + FileFragment[] tablets = new FileFragment[listTablets.size()]; + listTablets.toArray(tablets); + + return tablets; + } + + private class FetchRunner implements Runnable { + private final TaskAttemptContext ctx; + private final Fetcher fetcher; + private int maxRetryNum; + + public FetchRunner(TaskAttemptContext ctx, Fetcher fetcher) { + this.ctx = ctx; + this.fetcher = fetcher; + this.maxRetryNum = systemConf.getIntVar(TajoConf.ConfVars.SHUFFLE_FETCHER_READ_RETRY_MAX_NUM); + } + + @Override + public void run() { + int retryNum = 0; + int retryWaitTime = 1000; //sec + + try { // for releasing fetch latch + while(!context.isStopped() && retryNum < maxRetryNum) { + if (retryNum > 0) { + try { + Thread.sleep(retryWaitTime); + retryWaitTime = Math.min(10 * 1000, retryWaitTime * 2); // max 10 seconds + } catch (InterruptedException e) { + LOG.error(e); + } + LOG.warn("Retry on the fetch: " + fetcher.getURI() + " (" + retryNum + ")"); + } + try { + FileChunk fetched = fetcher.get(); + if (fetcher.getState() == TajoProtos.FetcherState.FETCH_FINISHED && fetched != null + && fetched.getFile() != null) { + if (fetched.fromRemote() == false) { + localChunks.add(fetched); + LOG.info("Add a new FileChunk to local chunk list"); + } + break; + } + } catch (Throwable e) { + LOG.error("Fetch failed: " + fetcher.getURI(), e); + } + retryNum++; + } + } finally { + if(fetcher.getState() == TajoProtos.FetcherState.FETCH_FINISHED){ + fetcherFinished(ctx); + } else { + if (retryNum == maxRetryNum) { + LOG.error("ERROR: the maximum retry (" + retryNum + ") on the fetch exceeded (" + fetcher.getURI() + ")"); + } + stopScriptExecutors(); + context.stop(); // retry task + ctx.getFetchLatch().countDown(); + } + } + } + } + + @VisibleForTesting + public static float adjustFetchProcess(int totalFetcher, int remainFetcher) { + if (totalFetcher > 0) { + return ((totalFetcher - remainFetcher) / (float) totalFetcher) * FETCHER_PROGRESS; + } else { + return 0.0f; + } + } + + private synchronized void fetcherFinished(TaskAttemptContext ctx) { + int fetcherSize = fetcherRunners.size(); + if(fetcherSize == 0) { + return; + } + + ctx.getFetchLatch().countDown(); + + int remainFetcher = (int) ctx.getFetchLatch().getCount(); + if (remainFetcher == 0) { + context.setFetcherProgress(FETCHER_PROGRESS); + } else { + context.setFetcherProgress(adjustFetchProcess(fetcherSize, remainFetcher)); + } + } + + private List getFetchRunners(TaskAttemptContext ctx, + List fetches) throws IOException { + + if (fetches.size() > 0) { + Path inputDir = executionBlockContext.getLocalDirAllocator(). + getLocalPathToRead(getTaskAttemptDir(ctx.getTaskId()).toString(), systemConf); + + int i = 0; + File storeDir; + File defaultStoreFile; + FileChunk storeChunk = null; + List runnerList = Lists.newArrayList(); + + for (FetchImpl f : fetches) { + storeDir = new File(inputDir.toString(), f.getName()); + if (!storeDir.exists()) { + if (!storeDir.mkdirs()) throw new IOException("Failed to create " + storeDir); + } + + for (URI uri : f.getURIs()) { + defaultStoreFile = new File(storeDir, "in_" + i); + InetAddress address = InetAddress.getByName(uri.getHost()); + + WorkerConnectionInfo conn = executionBlockContext.getWorkerContext().getConnectionInfo(); + if (NetUtils.isLocalAddress(address) && conn.getPullServerPort() == uri.getPort()) { + boolean hasError = false; + try { + LOG.info("Try to get local file chunk at local host"); + storeChunk = getLocalStoredFileChunk(uri, systemConf); + } catch (Throwable t) { + hasError = true; + } + + // When a range request is out of range, storeChunk will be NULL. This case is normal state. + // So, we should skip and don't need to create storeChunk. + if (storeChunk == null && !hasError) { + continue; + } + + if (storeChunk != null && storeChunk.getFile() != null && storeChunk.startOffset() > -1 + && hasError == false) { + storeChunk.setFromRemote(false); + } else { + storeChunk = new FileChunk(defaultStoreFile, 0, -1); + storeChunk.setFromRemote(true); + } + } else { + storeChunk = new FileChunk(defaultStoreFile, 0, -1); + storeChunk.setFromRemote(true); + } + + // If we decide that intermediate data should be really fetched from a remote host, storeChunk + // represents a complete file. Otherwise, storeChunk may represent a complete file or only a part of it + storeChunk.setEbId(f.getName()); + Fetcher fetcher = new Fetcher(systemConf, uri, storeChunk); + LOG.info("Create a new Fetcher with storeChunk:" + storeChunk.toString()); + runnerList.add(fetcher); + i++; + } + } + ctx.addFetchPhase(runnerList.size(), new File(inputDir.toString())); + return runnerList; + } else { + return Lists.newArrayList(); + } + } + + private FileChunk getLocalStoredFileChunk(URI fetchURI, TajoConf conf) throws IOException { + // Parse the URI + LOG.info("getLocalStoredFileChunk starts"); + final Map> params = new QueryStringDecoder(fetchURI.toString()).parameters(); + final List types = params.get("type"); + final List qids = params.get("qid"); + final List taskIdList = params.get("ta"); + final List stageIds = params.get("sid"); + final List partIds = params.get("p"); + final List offsetList = params.get("offset"); + final List lengthList = params.get("length"); + + if (types == null || stageIds == null || qids == null || partIds == null) { + LOG.error("Invalid URI - Required queryId, type, stage Id, and part id"); + return null; + } + + if (qids.size() != 1 && types.size() != 1 || stageIds.size() != 1) { + LOG.error("Invalid URI - Required qids, type, taskIds, stage Id, and part id"); + return null; + } + + String queryId = qids.get(0); + String shuffleType = types.get(0); + String sid = stageIds.get(0); + String partId = partIds.get(0); + + if (shuffleType.equals("r") && taskIdList == null) { + LOG.error("Invalid URI - For range shuffle, taskId is required"); + return null; + } + List taskIds = splitMaps(taskIdList); + + FileChunk chunk; + long offset = (offsetList != null && !offsetList.isEmpty()) ? Long.parseLong(offsetList.get(0)) : -1L; + long length = (lengthList != null && !lengthList.isEmpty()) ? Long.parseLong(lengthList.get(0)) : -1L; + + LOG.info("PullServer request param: shuffleType=" + shuffleType + ", sid=" + sid + ", partId=" + partId + + ", taskIds=" + taskIdList); + + // The working directory of Tajo worker for each query, including stage + String queryBaseDir = queryId.toString() + "/output" + "/" + sid + "/"; + + // If the stage requires a range shuffle + if (shuffleType.equals("r")) { + String ta = taskIds.get(0); + if (!executionBlockContext.getLocalDirAllocator().ifExists(queryBaseDir + ta + "/output/", conf)) { + LOG.warn("Range shuffle - file not exist"); + return null; + } + Path path = executionBlockContext.getLocalFS().makeQualified( + executionBlockContext.getLocalDirAllocator().getLocalPathToRead(queryBaseDir + ta + "/output/", conf)); + String startKey = params.get("start").get(0); + String endKey = params.get("end").get(0); + boolean last = params.get("final") != null; + + try { + chunk = TajoPullServerService.getFileChunks(path, startKey, endKey, last); + } catch (Throwable t) { + LOG.error("getFileChunks() throws exception"); + return null; + } + + // If the stage requires a hash shuffle or a scattered hash shuffle + } else if (shuffleType.equals("h") || shuffleType.equals("s")) { + int partParentId = HashShuffleAppenderManager.getPartParentId(Integer.parseInt(partId), (TajoConf) conf); + String partPath = queryBaseDir + "hash-shuffle/" + partParentId + "/" + partId; + if (!executionBlockContext.getLocalDirAllocator().ifExists(partPath, conf)) { + LOG.warn("Hash shuffle or Scattered hash shuffle - file not exist: " + partPath); + return null; + } + Path path = executionBlockContext.getLocalFS().makeQualified( + executionBlockContext.getLocalDirAllocator().getLocalPathToRead(partPath, conf)); + File file = new File(path.toUri()); + long startPos = (offset >= 0 && length >= 0) ? offset : 0; + long readLen = (offset >= 0 && length >= 0) ? length : file.length(); + + if (startPos >= file.length()) { + LOG.error("Start pos[" + startPos + "] great than file length [" + file.length() + "]"); + return null; + } + chunk = new FileChunk(file, startPos, readLen); + + } else { + LOG.error("Unknown shuffle type"); + return null; + } + + return chunk; + } + + private List splitMaps(List mapq) { + if (null == mapq) { + return null; + } + final List ret = new ArrayList(); + for (String s : mapq) { + Collections.addAll(ret, s.split(",")); + } + return ret; + } + + public static Path getTaskAttemptDir(TaskAttemptId quid) { + Path workDir = + StorageUtil.concatPath(ExecutionBlockContext.getBaseInputDir(quid.getTaskId().getExecutionBlockId()), + String.valueOf(quid.getTaskId().getId()), + String.valueOf(quid.getId())); + return workDir; + } +} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java index a87bd530ec..8072c1de48 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java @@ -43,32 +43,23 @@ import java.util.*; import java.util.concurrent.*; -public class TaskManager extends CompositeService implements EventHandler { +public class TaskManager extends AbstractService implements EventHandler { private static final Log LOG = LogFactory.getLog(TaskManager.class); private final TajoWorker.WorkerContext workerContext; private final Map executionBlockContextMap; private final Dispatcher dispatcher; + private final EventHandler rmEventHandler; private TajoConf tajoConf; - private TaskExecutor taskExecutor; private volatile boolean isStopped = false; - private TaskHistoryCleanerThread taskHistoryCleanerThread; - private EventHandler rmEventHandler; - public TaskManager(Dispatcher dispatcher, TajoWorker.WorkerContext workerContext, - EventHandler rmEventHandler) { - this(dispatcher, workerContext, null, rmEventHandler); - } - - public TaskManager(Dispatcher dispatcher, TajoWorker.WorkerContext workerContext, TaskExecutor taskExecutor, - EventHandler rmEventHandler) { + public TaskManager(Dispatcher dispatcher, TajoWorker.WorkerContext workerContext, EventHandler rmEventHandler) { super(TaskManager.class.getName()); this.dispatcher = dispatcher; this.workerContext = workerContext; this.executionBlockContextMap = Maps.newHashMap(); - this.taskExecutor = taskExecutor; this.rmEventHandler = rmEventHandler; } @@ -77,27 +68,12 @@ protected void serviceInit(Configuration conf) throws Exception { if (!(conf instanceof TajoConf)) { throw new IllegalArgumentException("Configuration must be a TajoConf instance"); } - this.tajoConf = (TajoConf)conf; - - if (taskExecutor == null) { - this.taskExecutor = new TaskExecutor(this, rmEventHandler); - } - addIfService(dispatcher); - addService(taskExecutor); - dispatcher.register(TaskExecutorEvent.EventType.class, taskExecutor); + this.tajoConf = (TajoConf)conf; dispatcher.register(TaskManagerEvent.EventType.class, this); - - taskHistoryCleanerThread = new TaskHistoryCleanerThread(); super.serviceInit(conf); } - @Override - protected void serviceStart() throws Exception { - taskHistoryCleanerThread.start(); - super.serviceStart(); - } - @Override protected void serviceStop() throws Exception { isStopped = true; @@ -106,91 +82,73 @@ protected void serviceStop() throws Exception { context.stop(); } executionBlockContextMap.clear(); - - if(taskHistoryCleanerThread != null) { - taskHistoryCleanerThread.interrupt(); - } - super.serviceStop(); } - protected EventHandler getEventHandler() { - return dispatcher.getEventHandler(); + protected Dispatcher getDispatcher() { + return dispatcher; } - @Override - public void handle(TaskManagerEvent event) { - LOG.info("======================== Processing " + event.getExecutionBlockId() + " of type " + event.getType()); - if (event instanceof ExecutionBlockStartEvent) { - ExecutionBlockStartEvent startEvent = (ExecutionBlockStartEvent) event; - ExecutionBlockContext context = executionBlockContextMap.get(startEvent.getExecutionBlockId()); - - if(context == null){ - TajoWorkerProtocol.RunExecutionBlockRequestProto request = startEvent.getRequestProto(); - try { - context = new ExecutionBlockContext(context.getConf(), - workerContext, - null, //TODO - new QueryContext(tajoConf, request.getQueryContext()), - request.getPlanJson(), - startEvent.getExecutionBlockId(), - new WorkerConnectionInfo(request.getQueryMaster()), - request.getShuffleType()); - - context.init(); - } catch (Throwable e) { - LOG.fatal(e.getMessage(), e); - throw new RuntimeException(e); - } - executionBlockContextMap.put(startEvent.getExecutionBlockId(), context); + protected void startExecutionBlock(ExecutionBlockStartEvent event){ + ExecutionBlockContext context = executionBlockContextMap.get(event.getExecutionBlockId()); + + if(context == null){ + TajoWorkerProtocol.RunExecutionBlockRequestProto request = event.getRequestProto(); + try { + context = new ExecutionBlockContext(workerContext, null, request); + + context.init(); + } catch (Throwable e) { + LOG.fatal(e.getMessage(), e); + throw new RuntimeException(e); } - } else if (event instanceof ExecutionBlockStopEvent) { - ExecutionBlockStopEvent stopEvent = (ExecutionBlockStopEvent) event; - ExecutionBlockContext executionBlockContext = executionBlockContextMap.remove(stopEvent.getExecutionBlockId()); - if(executionBlockContext != null){ - try { - executionBlockContext.getSharedResource().releaseBroadcastCache(event.getExecutionBlockId()); - executionBlockContext.sendShuffleReport(); - workerContext.getTaskHistoryWriter().flushTaskHistories(); - } catch (Exception e) { - LOG.fatal(e.getMessage(), e); - throw new RuntimeException(e); - } finally { - executionBlockContext.stop(); + executionBlockContextMap.put(event.getExecutionBlockId(), context); + } + } + + protected void stopExecutionBlock(ExecutionBlockStopEvent event) { + ExecutionBlockContext executionBlockContext = executionBlockContextMap.remove(event.getExecutionBlockId()); + if(executionBlockContext != null){ + try { + executionBlockContext.getSharedResource().releaseBroadcastCache(event.getExecutionBlockId()); + executionBlockContext.sendShuffleReport(); + workerContext.getTaskHistoryWriter().flushTaskHistories(); + } catch (Exception e) { + LOG.fatal(e.getMessage(), e); + throw new RuntimeException(e); + } finally { + executionBlockContext.stop(); /* cleanup intermediate files */ - List cleanupList = stopEvent.getCleanupList().getExecutionBlockIdList(); - for (TajoIdProtos.ExecutionBlockIdProto ebId : cleanupList) { - String inputDir = ExecutionBlockContext.getBaseInputDir(new ExecutionBlockId(ebId)).toString(); - workerContext.cleanup(inputDir); - String outputDir = ExecutionBlockContext.getBaseOutputDir(new ExecutionBlockId(ebId)).toString(); - workerContext.cleanup(outputDir); - } + List cleanupList = event.getCleanupList().getExecutionBlockIdList(); + for (TajoIdProtos.ExecutionBlockIdProto ebId : cleanupList) { + String inputDir = ExecutionBlockContext.getBaseInputDir(new ExecutionBlockId(ebId)).toString(); + workerContext.cleanup(inputDir); + String outputDir = ExecutionBlockContext.getBaseOutputDir(new ExecutionBlockId(ebId)).toString(); + workerContext.cleanup(outputDir); } } - LOG.info("Stopped execution block:" + event.getExecutionBlockId()); } + LOG.info("Stopped execution block:" + event.getExecutionBlockId()); } - protected ExecutionBlockContext getExecutionBlockContext(ExecutionBlockId executionBlockId) { - return executionBlockContextMap.get(executionBlockId); - } - - - /*public Collection getTaskRunners() { - return Collections.unmodifiableCollection(taskRunnerMap.values()); - } + @Override + public void handle(TaskManagerEvent event) { + LOG.info("======================== Processing " + event.getExecutionBlockId() + " of type " + event.getType()); - public Collection getExecutionBlockHistories() { - return Collections.unmodifiableCollection(taskRunnerHistoryMap.values()); - } + if (event instanceof ExecutionBlockStartEvent) { - public TaskRunnerHistory getExcutionBlockHistoryByTaskRunnerId(String taskRunnerId) { - return taskRunnerHistoryMap.get(taskRunnerId); + //trigger event from NodeResourceManager + startExecutionBlock((ExecutionBlockStartEvent) event); + } else if (event instanceof ExecutionBlockStopEvent) { + //trigger event from QueryMaster + rmEventHandler.handle(new NodeStatusEvent(NodeStatusEvent.EventType.FLUSH_REPORTS)); + stopExecutionBlock((ExecutionBlockStopEvent) event); + } } - public TaskRunner getTaskRunner(String taskRunnerId) { - return taskRunnerMap.get(taskRunnerId); + protected ExecutionBlockContext getExecutionBlockContext(ExecutionBlockId executionBlockId) { + return executionBlockContextMap.get(executionBlockId); } public Task getTaskByTaskAttemptId(TaskAttemptId taskAttemptId) { @@ -201,53 +159,23 @@ public Task getTaskByTaskAttemptId(TaskAttemptId taskAttemptId) { return null; } - public TaskHistory getTaskHistoryByTaskAttemptId(TaskAttemptId quAttemptId) { - synchronized (taskRunnerHistoryMap) { - for (TaskRunnerHistory history : taskRunnerHistoryMap.values()) { - TaskHistory taskHistory = history.getTaskHistory(quAttemptId); - if (taskHistory != null) return taskHistory; - } + public List getTaskHistories(ExecutionBlockId executionblockId) throws IOException { + List histories = new ArrayList(); + ExecutionBlockContext context = executionBlockContextMap.get(executionblockId); + if (context != null) { + histories.addAll(context.getTaskHistories().values()); } - - return null; + //TODO get List from HistoryReader + return histories; } - */ - - class TaskHistoryCleanerThread extends Thread { - //TODO if history size is large, the historyMap should remove immediately - public void run() { - int expireIntervalTime = tajoConf.getIntVar(TajoConf.ConfVars.WORKER_HISTORY_EXPIRE_PERIOD); - LOG.info("FinishedQueryMasterTaskCleanThread started: expire interval minutes = " + expireIntervalTime); - while(!isStopped) { - try { - Thread.sleep(60 * 1000); - } catch (InterruptedException e) { - break; - } - try { - long expireTime = System.currentTimeMillis() - expireIntervalTime * 60 * 1000l; - cleanExpiredFinishedQueryMasterTask(expireTime); - } catch (Exception e) { - LOG.error(e.getMessage(), e); - } - } - } - private void cleanExpiredFinishedQueryMasterTask(long expireTime) { -// synchronized(taskRunnerHistoryMap) { -// List expiredIds = new ArrayList(); -// for(Map.Entry entry: taskRunnerHistoryMap.entrySet()) { -// /* If a task runner are abnormal termination, the finished time will be zero. */ -// long finishedTime = Math.max(entry.getValue().getStartTime(), entry.getValue().getFinishTime()); -// if(finishedTime < expireTime) { -// expiredIds.add(entry.getKey()); -// } -// } -// -// for(String eachId: expiredIds) { -// taskRunnerHistoryMap.remove(eachId); -// } -// } + public TaskHistory getTaskHistory(TaskId taskId) { + TaskHistory history = null; + ExecutionBlockContext context = executionBlockContextMap.get(taskId.getExecutionBlockId()); + if (context != null) { + history = context.getTaskHistories().get(taskId); } + //TODO get TaskHistory from HistoryReader + return history; } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunner.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunner.java index aff2daf62a..207b47e9f8 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunner.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunner.java @@ -43,6 +43,7 @@ /** * The driver class for Tajo Task processing. */ +@Deprecated public class TaskRunner extends AbstractService { /** class logger */ private static final Log LOG = LogFactory.getLog(TaskRunner.class); diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerHistory.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerHistory.java index 5c97ba81d8..16d32d443d 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerHistory.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerHistory.java @@ -36,6 +36,7 @@ /** * The history class for TaskRunner processing. */ +@Deprecated public class TaskRunnerHistory implements ProtoObject { private Service.STATE state; diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java index 734a8a5b4b..d18a26295e 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java @@ -36,6 +36,7 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicBoolean; +@Deprecated public class TaskRunnerManager extends CompositeService implements EventHandler { private static final Log LOG = LogFactory.getLog(TaskRunnerManager.class); @@ -154,14 +155,7 @@ public void handle(TaskRunnerEvent event) { if(context == null){ try { - context = new ExecutionBlockContext(getTajoConf(), - getWorkerContext(), - this, - startEvent.getQueryContext(), - startEvent.getPlan(), - startEvent.getExecutionBlockId(), - startEvent.getQueryMaster(), - startEvent.getShuffleType()); + context = new ExecutionBlockContext(getWorkerContext(), this, startEvent.getRequest()); context.init(); } catch (Throwable e) { LOG.fatal(e.getMessage(), e); @@ -170,7 +164,7 @@ public void handle(TaskRunnerEvent event) { executionBlockContextMap.put(event.getExecutionBlockId(), context); } - TaskRunner taskRunner = new TaskRunner(context, startEvent.getContainerId()); + TaskRunner taskRunner = new TaskRunner(context, startEvent.getRequest().getContainerId()); LOG.info("Start TaskRunner:" + taskRunner.getId()); taskRunnerMap.put(taskRunner.getId(), taskRunner); taskRunnerHistoryMap.put(taskRunner.getId(), taskRunner.getHistory()); diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceAllocateEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceAllocateEvent.java index 2f411e8c5c..9a1c106865 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceAllocateEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceAllocateEvent.java @@ -24,7 +24,7 @@ import static org.apache.tajo.ipc.TajoWorkerProtocol.BatchAllocationRequestProto; import static org.apache.tajo.ipc.TajoWorkerProtocol.BatchAllocationResponseProto; -public class NodeResourceAllocateEvent extends NodeResourceManagerEvent { +public class NodeResourceAllocateEvent extends NodeResourceEvent { private BatchAllocationRequestProto request; private RpcCallback callback; diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceDeallocateEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceDeallocateEvent.java index a298d77e20..31d9229ad4 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceDeallocateEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceDeallocateEvent.java @@ -21,7 +21,7 @@ import org.apache.tajo.TajoProtos; import org.apache.tajo.resource.NodeResource; -public class NodeResourceDeallocateEvent extends NodeResourceManagerEvent { +public class NodeResourceDeallocateEvent extends NodeResourceEvent { private NodeResource resource; diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceManagerEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceEvent.java similarity index 80% rename from tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceManagerEvent.java rename to tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceEvent.java index bcb34486c0..69c1d0c1e4 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceManagerEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceEvent.java @@ -19,16 +19,14 @@ package org.apache.tajo.worker.event; import org.apache.hadoop.yarn.event.AbstractEvent; -import org.apache.tajo.ExecutionBlockId; -import org.apache.tajo.resource.NodeResource; -public class NodeResourceManagerEvent extends AbstractEvent { +public class NodeResourceEvent extends AbstractEvent { public enum EventType { ALLOCATE, DEALLOCATE } - public NodeResourceManagerEvent(EventType eventType) { + public NodeResourceEvent(EventType eventType) { super(eventType); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeStatusEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeStatusEvent.java index 58ab74af0a..ec604fe76e 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeStatusEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeStatusEvent.java @@ -22,19 +22,13 @@ import org.apache.tajo.resource.NodeResource; public class NodeStatusEvent extends AbstractEvent { - private final NodeResource resource; public enum EventType { REPORT_RESOURCE, FLUSH_REPORTS } - public NodeStatusEvent(EventType eventType, NodeResource resource) { + public NodeStatusEvent(EventType eventType) { super(eventType); - this.resource = resource; - } - - public NodeResource getResource() { - return resource; } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerEvent.java index aac8973b81..717525197d 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerEvent.java @@ -21,6 +21,7 @@ import org.apache.hadoop.yarn.event.AbstractEvent; import org.apache.tajo.ExecutionBlockId; +@Deprecated public class TaskRunnerEvent extends AbstractEvent { public enum EventType { START, diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerStartEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerStartEvent.java index 908afa2293..9406794254 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerStartEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerStartEvent.java @@ -20,48 +20,20 @@ import org.apache.tajo.ExecutionBlockId; import org.apache.tajo.engine.query.QueryContext; +import org.apache.tajo.ipc.TajoWorkerProtocol; import org.apache.tajo.master.cluster.WorkerConnectionInfo; import org.apache.tajo.plan.serder.PlanProto; - +@Deprecated public class TaskRunnerStartEvent extends TaskRunnerEvent { - private final QueryContext queryContext; - private final WorkerConnectionInfo queryMaster; - private final String containerId; - private final String plan; - private final PlanProto.ShuffleType shuffleType; - - public TaskRunnerStartEvent(WorkerConnectionInfo queryMaster, - ExecutionBlockId executionBlockId, - String containerId, - QueryContext context, - String plan, - PlanProto.ShuffleType shuffleType) { - super(EventType.START, executionBlockId); - this.queryMaster = queryMaster; - this.containerId = containerId; - this.queryContext = context; - this.plan = plan; - this.shuffleType = shuffleType; - } - - public WorkerConnectionInfo getQueryMaster() { - return queryMaster; - } - - public String getContainerId() { - return containerId; - } - - public QueryContext getQueryContext() { - return queryContext; - } + private final TajoWorkerProtocol.RunExecutionBlockRequestProto request; - public String getPlan() { - return plan; + public TaskRunnerStartEvent(TajoWorkerProtocol.RunExecutionBlockRequestProto request) { + super(EventType.START, new ExecutionBlockId(request.getExecutionBlockId())); + this.request = request; } - public PlanProto.ShuffleType getShuffleType() { - return shuffleType; + public TajoWorkerProtocol.RunExecutionBlockRequestProto getRequest() { + return request; } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerStopEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerStopEvent.java index c8ec20d138..297f30c037 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerStopEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskRunnerStopEvent.java @@ -20,6 +20,7 @@ import org.apache.tajo.ExecutionBlockId; +@Deprecated public class TaskRunnerStopEvent extends TaskRunnerEvent { public TaskRunnerStopEvent(ExecutionBlockId executionBlockId) { diff --git a/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java index 2384b19620..0cec3dac49 100644 --- a/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java @@ -19,6 +19,7 @@ package org.apache.tajo.querymaster; import com.google.common.collect.Lists; +import org.apache.hadoop.fs.LocalDirAllocator; import org.apache.hadoop.yarn.event.AsyncDispatcher; import org.apache.hadoop.yarn.event.Event; import org.apache.tajo.*; @@ -33,15 +34,25 @@ import org.apache.tajo.engine.planner.global.MasterPlan; import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.engine.query.TaskRequestImpl; +import org.apache.tajo.ipc.QueryCoordinatorProtocol; +import org.apache.tajo.ipc.TajoWorkerProtocol; +import org.apache.tajo.master.cluster.WorkerConnectionInfo; import org.apache.tajo.master.event.*; import org.apache.tajo.plan.LogicalOptimizer; import org.apache.tajo.plan.LogicalPlan; import org.apache.tajo.plan.LogicalPlanner; import org.apache.tajo.plan.serder.PlanProto; +import org.apache.tajo.service.ServiceTracker; import org.apache.tajo.session.Session; +import org.apache.tajo.storage.HashShuffleAppenderManager; import org.apache.tajo.util.CommonTestingUtil; +import org.apache.tajo.util.history.HistoryReader; +import org.apache.tajo.util.history.HistoryWriter; +import org.apache.tajo.util.metrics.TajoSystemMetrics; import org.apache.tajo.worker.ExecutionBlockContext; import org.apache.tajo.worker.LegacyTaskImpl; +import org.apache.tajo.worker.TajoWorker; +import org.apache.tajo.worker.TaskRunnerManager; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; @@ -229,7 +240,7 @@ public void testKillTask() throws Throwable { QueryId qid = LocalTajoTestingUtility.newQueryId(); ExecutionBlockId eid = QueryIdFactory.newExecutionBlockId(qid, 1); TaskId tid = QueryIdFactory.newTaskId(eid); - TajoConf conf = new TajoConf(); + final TajoConf conf = new TajoConf(); TaskRequestImpl taskRequest = new TaskRequestImpl(); taskRequest.set(null, new ArrayList(), @@ -237,8 +248,27 @@ public void testKillTask() throws Throwable { taskRequest.setInterQuery(); TaskAttemptId attemptId = new TaskAttemptId(tid, 1); + WorkerConnectionInfo queryMaster = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); + TajoWorkerProtocol.RunExecutionBlockRequestProto.Builder + requestProto = TajoWorkerProtocol.RunExecutionBlockRequestProto.newBuilder(); + + requestProto.setExecutionBlockId(eid.getProto()) + .setQueryMaster(queryMaster.getProto()) + .setNodeId(queryMaster.getHost()+":" + queryMaster.getQueryMasterPort()) + .setContainerId("test") + .setQueryContext(new QueryContext(conf).getProto()) + .setPlanJson("test") + .setShuffleType(PlanProto.ShuffleType.HASH_SHUFFLE); + + TajoWorker.WorkerContext workerContext = new MockWorkerContext() { + @Override + public TajoConf getConf() { + return conf; + } + }; + ExecutionBlockContext context = - new ExecutionBlockContext(conf, null, null, new QueryContext(conf), null, eid, null, null); + new ExecutionBlockContext(workerContext, null, requestProto.build()); org.apache.tajo.worker.Task task = new LegacyTaskImpl("test", CommonTestingUtil.getTestDir(), attemptId, conf, context, taskRequest); @@ -270,4 +300,94 @@ protected void dispatch(Event event) { super.dispatch(event); } } + + abstract class MockWorkerContext implements TajoWorker.WorkerContext { + + @Override + public QueryMaster getQueryMaster() { + return null; + } + + public abstract TajoConf getConf(); + + @Override + public ServiceTracker getServiceTracker() { + return null; + } + + @Override + public QueryMasterManagerService getQueryMasterManagerService() { + return null; + } + + @Override + public TaskRunnerManager getTaskRunnerManager() { + return null; + } + + @Override + public CatalogService getCatalog() { + return null; + } + + @Override + public WorkerConnectionInfo getConnectionInfo() { + return null; + } + + @Override + public String getWorkerName() { + return null; + } + + @Override + public LocalDirAllocator getLocalDirAllocator() { + return null; + } + + @Override + public QueryCoordinatorProtocol.ClusterResourceSummary getClusterResource() { + return null; + } + + @Override + public TajoSystemMetrics getWorkerSystemMetrics() { + return null; + } + + @Override + public HashShuffleAppenderManager getHashShuffleAppenderManager() { + return null; + } + + @Override + public HistoryWriter getTaskHistoryWriter() { + return null; + } + + @Override + public HistoryReader getHistoryReader() { + return null; + } + + @Override + public void cleanup(String strPath) { + + } + + @Override + public void cleanupTemporalDirectories() { + + } + + @Override + public void setClusterResource(QueryCoordinatorProtocol.ClusterResourceSummary clusterResource) { + + } + + @Override + public void setNumClusterNodes(int numClusterNodes) { + + } + } } diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeNodeResourceManager.java b/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeNodeResourceManager.java new file mode 100644 index 0000000000..76e32163e6 --- /dev/null +++ b/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeNodeResourceManager.java @@ -0,0 +1,54 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.worker; + +import org.apache.hadoop.yarn.event.Dispatcher; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.tajo.ipc.TajoWorkerProtocol; +import org.apache.tajo.resource.NodeResource; + +public class MockNodeNodeResourceManager extends NodeResourceManager { + + volatile boolean enableTaskHandlerEvent; + + public MockNodeNodeResourceManager(Dispatcher dispatcher, EventHandler taskEventHandler) { + super(dispatcher, taskEventHandler); + } + + @Override + protected void startExecutionBlock(TajoWorkerProtocol.RunExecutionBlockRequestProto request) { + if(enableTaskHandlerEvent) { + super.startExecutionBlock(request); + } + } + + @Override + protected void startTask(TajoWorkerProtocol.TaskRequestProto request, NodeResource resource) { + if(enableTaskHandlerEvent) { + super.startTask(request, resource); + } + } + + /** + * skip task execution and deallocation for testing + * */ + public void setTaskHandlerEvent(boolean flag) { + enableTaskHandlerEvent = flag; + } +} diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskExecutor.java b/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskExecutor.java new file mode 100644 index 0000000000..1748546b6b --- /dev/null +++ b/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskExecutor.java @@ -0,0 +1,128 @@ +/** + * 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.tajo.worker; + +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.tajo.TajoProtos; +import org.apache.tajo.TaskAttemptId; +import org.apache.tajo.catalog.statistics.TableStats; +import org.apache.tajo.ipc.TajoWorkerProtocol; + +import java.io.IOException; + +public class MockTaskExecutor extends TaskExecutor { + + public MockTaskExecutor(TaskManager taskManager, EventHandler rmEventHandler) { + super(taskManager, rmEventHandler); + } + + @Override + protected Task createTask(final ExecutionBlockContext context, TajoWorkerProtocol.TaskRequestProto taskRequest) { + final TaskAttemptId taskAttemptId = new TaskAttemptId(taskRequest.getId()); + final TaskAttemptContext taskAttemptContext = new TaskAttemptContext(null, context, taskAttemptId, null, null) { + private TajoProtos.TaskAttemptState state; + + @Override + public TajoProtos.TaskAttemptState getState() { + return state; + } + + @Override + public void setState(TajoProtos.TaskAttemptState state) { + this.state = state; + } + }; + + return new Task() { + @Override + public void init() throws IOException { + + } + + @Override + public void fetch() { + + } + + @Override + public void run() throws Exception { + taskAttemptContext.stop(); + taskAttemptContext.setProgress(1.0f); + taskAttemptContext.setState(TajoProtos.TaskAttemptState.TA_SUCCEEDED); + } + + @Override + public void kill() { + + } + + @Override + public void abort() { + + } + + @Override + public void cleanup() { + + } + + @Override + public boolean hasFetchPhase() { + return false; + } + + @Override + public boolean isProgressChanged() { + return false; + } + + @Override + public boolean isStopped() { + return taskAttemptContext.isStopped(); + } + + @Override + public void updateProgress() { + + } + + @Override + public TaskAttemptContext getTaskContext() { + return taskAttemptContext; + } + + @Override + public ExecutionBlockContext getExecutionBlockContext() { + return context; + } + + @Override + public TajoWorkerProtocol.TaskStatusProto getReport() { + TajoWorkerProtocol.TaskStatusProto.Builder builder = TajoWorkerProtocol.TaskStatusProto.newBuilder(); + builder.setWorkerName("localhost:0"); + builder.setId(taskAttemptContext.getTaskId().getProto()) + .setProgress(taskAttemptContext.getProgress()) + .setState(taskAttemptContext.getState()); + + builder.setInputStats(new TableStats().getProto()); + return builder.build(); + } + }; + } +} diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java index 49e9fa8207..0b088b031d 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java @@ -19,6 +19,8 @@ package org.apache.tajo.worker; import com.google.common.collect.Lists; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.service.CompositeService; import org.apache.hadoop.yarn.event.AsyncDispatcher; import org.apache.tajo.*; import org.apache.tajo.conf.TajoConf; @@ -42,9 +44,14 @@ import static org.apache.tajo.ipc.TajoWorkerProtocol.*; public class TestNodeResourceManager { - private NodeResourceManager resourceManager; - private MockNodeStatusUpdater statusUpdater; + private MockNodeNodeResourceManager resourceManager; + private NodeStatusUpdater statusUpdater; + private TaskManager taskManager; + private TaskExecutor taskExecutor; private AsyncDispatcher dispatcher; + private AsyncDispatcher taskDispatcher; + + private CompositeService service; private int taskMemory; private TajoConf conf; @@ -61,34 +68,40 @@ public void setup() { conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_DISK_PARALLEL_NUM, 1); dispatcher = new AsyncDispatcher(); - dispatcher.init(conf); - dispatcher.start(); - - AsyncDispatcher taskDispatcher = new AsyncDispatcher(); - TaskManager taskManager = new TaskManager(taskDispatcher, null, dispatcher.getEventHandler()); - taskManager.init(conf); - taskManager.start(); - - resourceManager = new NodeResourceManager(dispatcher, taskManager); - resourceManager.init(conf); - resourceManager.start(); + taskDispatcher = new AsyncDispatcher(); + taskManager = new TaskManager(taskDispatcher, null, dispatcher.getEventHandler()); + taskExecutor = new MockTaskExecutor(taskManager, dispatcher.getEventHandler()); + resourceManager = new MockNodeNodeResourceManager(dispatcher, taskDispatcher.getEventHandler()); WorkerConnectionInfo worker = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); statusUpdater = new MockNodeStatusUpdater(new CountDownLatch(0), worker, resourceManager); - statusUpdater.init(conf); - statusUpdater.start(); + + service = new CompositeService("MockService") { + @Override + protected void serviceInit(Configuration conf) throws Exception { + addIfService(dispatcher); + addIfService(taskDispatcher); + addIfService(taskManager); + addIfService(taskExecutor); + addIfService(resourceManager); + addIfService(statusUpdater); + super.serviceInit(conf); + } + }; + + service.init(conf); + service.start(); } @After public void tearDown() { - resourceManager.stop(); - statusUpdater.stop(); - dispatcher.stop(); + service.stop(); } @Test public void testNodeResourceAllocateEvent() throws Exception { int requestSize = 4; + resourceManager.setTaskHandlerEvent(false); //skip task execution CallFuture callFuture = new CallFuture(); BatchAllocationRequestProto.Builder requestProto = BatchAllocationRequestProto.newBuilder(); @@ -102,8 +115,8 @@ public void testNodeResourceAllocateEvent() throws Exception { BatchAllocationResponseProto responseProto = callFuture.get(); assertNotEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); + // allocated all assertEquals(0, responseProto.getCancellationTaskCount()); - assertEquals(requestSize, resourceManager.getAllocatedSize()); } @@ -111,6 +124,7 @@ public void testNodeResourceAllocateEvent() throws Exception { public void testNodeResourceCancellation() throws Exception { int requestSize = conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES); int overSize = 10; + resourceManager.setTaskHandlerEvent(false); //skip task execution CallFuture callFuture = new CallFuture(); BatchAllocationRequestProto.Builder requestProto = BatchAllocationRequestProto.newBuilder(); @@ -124,12 +138,12 @@ public void testNodeResourceCancellation() throws Exception { BatchAllocationResponseProto responseProto = callFuture.get(); assertEquals(overSize, responseProto.getCancellationTaskCount()); - assertEquals(requestSize, resourceManager.getAllocatedSize()); } @Test public void testNodeResourceDeallocateEvent() throws Exception { int requestSize = 4; + resourceManager.setTaskHandlerEvent(false); //skip task execution CallFuture callFuture = new CallFuture(); BatchAllocationRequestProto.Builder requestProto = BatchAllocationRequestProto.newBuilder(); @@ -144,15 +158,13 @@ public void testNodeResourceDeallocateEvent() throws Exception { BatchAllocationResponseProto responseProto = callFuture.get(); assertNotEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); assertEquals(0, responseProto.getCancellationTaskCount()); - assertEquals(requestSize, resourceManager.getAllocatedSize()); //deallocate -// for(TaskAllocationRequestProto allocationRequestProto : requestProto.getTaskRequestList()) { -// // direct invoke handler for testing -// resourceManager.handle(new NodeResourceDeallocateEvent(allocationRequestProto.getResource())); -// } - Thread.sleep(1000); - assertEquals(0, resourceManager.getAllocatedSize()); + for(TaskAllocationRequestProto allocationRequestProto : requestProto.getTaskRequestList()) { + // direct invoke handler for testing + resourceManager.handle(new NodeResourceDeallocateEvent(allocationRequestProto.getResource())); + } + assertEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); } @@ -160,6 +172,8 @@ public void testNodeResourceDeallocateEvent() throws Exception { public void testParallelRequest() throws Exception { final int parallelCount = conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES) * 2; final int taskSize = 100000; + resourceManager.setTaskHandlerEvent(true); + final AtomicInteger totalComplete = new AtomicInteger(); final AtomicInteger totalCanceled = new AtomicInteger(); @@ -193,7 +207,6 @@ public void run() { totalCanceled.addAndGet(proto.getCancellationTaskCount()); } else { complete++; - // dispatcher.getEventHandler().handle(new NodeResourceDeallocateEvent(task.getResource())); } } catch (Exception e) { fail(e.getMessage()); diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java index 134aef81e7..0139088dad 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java @@ -105,11 +105,26 @@ public void testResourceReport() throws Exception { statusUpdater.init(conf); statusUpdater.start(); + assertEquals(0, statusUpdater.getQueueSize()); for (int i = 0; i < statusUpdater.getQueueingLimit(); i++) { - dispatcher.getEventHandler().handle(new NodeStatusEvent(NodeStatusEvent.EventType.REPORT_RESOURCE, - resourceManager.getAvailableResource())); + dispatcher.getEventHandler().handle(new NodeStatusEvent(NodeStatusEvent.EventType.REPORT_RESOURCE)); } barrier.await(); assertEquals(0, statusUpdater.getQueueSize()); } + + @Test(timeout = 20000) + public void testFlushResourceReport() throws Exception { + CountDownLatch barrier = new CountDownLatch(2); + WorkerConnectionInfo worker = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); + statusUpdater = new MockNodeStatusUpdater(barrier, worker, resourceManager); + statusUpdater.init(conf); + statusUpdater.start(); + + assertEquals(0, statusUpdater.getQueueSize()); + dispatcher.getEventHandler().handle(new NodeStatusEvent(NodeStatusEvent.EventType.FLUSH_REPORTS)); + + barrier.await(); + assertEquals(0, statusUpdater.getQueueSize()); + } } From b913c88699f5ba6fc13a9f1ee0b65c2b1fba68fc Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Fri, 29 May 2015 17:32:09 +0900 Subject: [PATCH 17/21] add more test case --- .../tajo/worker/ExecutionBlockContext.java | 12 +- .../org/apache/tajo/worker/TaskContainer.java | 5 +- .../org/apache/tajo/worker/TaskExecutor.java | 2 + .../org/apache/tajo/worker/TaskManager.java | 66 ++-- ...ceManager.java => MockExecutionBlock.java} | 30 +- .../tajo/worker/MockNodeResourceManager.java | 96 ++++++ .../apache/tajo/worker/MockTaskExecutor.java | 15 +- .../apache/tajo/worker/MockTaskManager.java | 59 ++++ .../apache/tajo/worker/MockWorkerContext.java | 129 +++++++ .../tajo/worker/TestNodeResourceManager.java | 75 ++-- .../tajo/worker/TestNodeStatusUpdater.java | 8 +- .../apache/tajo/worker/TestTaskExecutor.java | 320 ++++++++++++++++++ .../apache/tajo/worker/TestTaskManager.java | 175 ++++++++++ 13 files changed, 887 insertions(+), 105 deletions(-) rename tajo-core/src/test/java/org/apache/tajo/worker/{MockNodeNodeResourceManager.java => MockExecutionBlock.java} (54%) create mode 100644 tajo-core/src/test/java/org/apache/tajo/worker/MockNodeResourceManager.java create mode 100644 tajo-core/src/test/java/org/apache/tajo/worker/MockTaskManager.java create mode 100644 tajo-core/src/test/java/org/apache/tajo/worker/MockWorkerContext.java create mode 100644 tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java create mode 100644 tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java b/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java index f5e256ce38..9e4a60f758 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/ExecutionBlockContext.java @@ -132,7 +132,8 @@ public void init() throws Throwable { UserGroupInformation.setConfiguration(systemConf); // TODO - 'load credential' should be implemented // Getting taskOwner - UserGroupInformation taskOwner = UserGroupInformation.createRemoteUser(systemConf.getVar(TajoConf.ConfVars.USERNAME)); + UserGroupInformation + taskOwner = UserGroupInformation.createRemoteUser(systemConf.getVar(TajoConf.ConfVars.USERNAME)); // initialize DFS and LocalFileSystems this.taskOwner = taskOwner; @@ -378,7 +379,6 @@ private void sendHashShuffleReport(ExecutionBlockId ebId) throws Exception { protected class Reporter { private Thread reporterThread; - private AtomicBoolean reporterStop = new AtomicBoolean(); private static final int PROGRESS_INTERVAL = 1000; private static final int MAX_RETRIES = 10; @@ -397,7 +397,7 @@ Runnable createReporterThread() { int remainingRetries = MAX_RETRIES; @Override public void run() { - while (!reporterStop.get() && !Thread.interrupted()) { + while (!isStopped() && !Thread.interrupted()) { try { Interface masterStub = getStub(); @@ -423,7 +423,7 @@ public void run() { throw new RuntimeException(t); } } finally { - if (remainingRetries > 0 && !reporterStop.get()) { + if (remainingRetries > 0 && !isStopped()) { synchronized (reporterThread) { try { reporterThread.wait(PROGRESS_INTERVAL); @@ -438,10 +438,6 @@ public void run() { } public void stop() throws InterruptedException { - if (reporterStop.getAndSet(true)) { - return; - } - if (reporterThread != null) { // Intent of the lock is to not send an interupt in the middle of an // umbilical.ping or umbilical.statusUpdate diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskContainer.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskContainer.java index 1d2daa6c1d..2576726840 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskContainer.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskContainer.java @@ -44,16 +44,13 @@ public void run() { try { task = executor.getNextTask(); - if (task.getExecutionBlockContext() != null) { - task.getExecutionBlockContext().getWorkerContext().getWorkerSystemMetrics().counter("query", "task").inc(); - } + task.getExecutionBlockContext().getWorkerContext().getWorkerSystemMetrics().counter("query", "task").inc(); if (LOG.isDebugEnabled()) { LOG.debug(sequenceId + TaskContainer.class.getSimpleName() + " got task:" + task.getTaskContext().getTaskId()); } - TaskAttemptContext taskAttemptContext = task.getTaskContext(); if (taskAttemptContext.isStopped()) return; diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java index 9235404d6d..87f18bbc7a 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java @@ -172,6 +172,8 @@ public void handle(TaskExecutorEvent event) { } taskQueue.put(task); runningTasks.incrementAndGet(); + context.getWorkerContext().getWorkerSystemMetrics() + .histogram("tasks", "running").update(runningTasks.get()); } else { LOG.warn("Release duplicate task resource: " + startEvent.getAllocatedResource()); stopTask(startEvent.getTaskId()); diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java index 8072c1de48..bbf188e14b 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java @@ -19,12 +19,10 @@ package org.apache.tajo.worker; import com.google.common.collect.Maps; -import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.service.AbstractService; -import org.apache.hadoop.service.CompositeService; import org.apache.hadoop.yarn.event.Dispatcher; import org.apache.hadoop.yarn.event.EventHandler; import org.apache.tajo.ExecutionBlockId; @@ -32,16 +30,11 @@ import org.apache.tajo.TaskAttemptId; import org.apache.tajo.TaskId; import org.apache.tajo.conf.TajoConf; -import org.apache.tajo.conf.TajoConf.ConfVars; -import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.ipc.TajoWorkerProtocol; -import org.apache.tajo.master.cluster.WorkerConnectionInfo; -import org.apache.tajo.resource.NodeResource; import org.apache.tajo.worker.event.*; import java.io.IOException; import java.util.*; -import java.util.concurrent.*; public class TaskManager extends AbstractService implements EventHandler { private static final Log LOG = LogFactory.getLog(TaskManager.class); @@ -52,7 +45,6 @@ public class TaskManager extends AbstractService implements EventHandler cleanupList = event.getCleanupList().getExecutionBlockIdList(); - for (TajoIdProtos.ExecutionBlockIdProto ebId : cleanupList) { + for (TajoIdProtos.ExecutionBlockIdProto ebId : cleanupList.getExecutionBlockIdList()) { String inputDir = ExecutionBlockContext.getBaseInputDir(new ExecutionBlockId(ebId)).toString(); workerContext.cleanup(inputDir); String outputDir = ExecutionBlockContext.getBaseOutputDir(new ExecutionBlockId(ebId)).toString(); workerContext.cleanup(outputDir); } } + LOG.info("Stopped execution block:" + context.getExecutionBlockId()); } - LOG.info("Stopped execution block:" + event.getExecutionBlockId()); } @Override @@ -138,12 +128,18 @@ public void handle(TaskManagerEvent event) { if (event instanceof ExecutionBlockStartEvent) { - //trigger event from NodeResourceManager - startExecutionBlock((ExecutionBlockStartEvent) event); + //receive event from NodeResourceManager + if(!executionBlockContextMap.containsKey(event.getExecutionBlockId())) { + ExecutionBlockContext context = createExecutionBlock(((ExecutionBlockStartEvent) event).getRequestProto()); + executionBlockContextMap.put(context.getExecutionBlockId(), context); + } else { + LOG.warn("Already initialized ExecutionBlock: " + event.getExecutionBlockId()); + } } else if (event instanceof ExecutionBlockStopEvent) { - //trigger event from QueryMaster + //receive event from QueryMaster rmEventHandler.handle(new NodeStatusEvent(NodeStatusEvent.EventType.FLUSH_REPORTS)); - stopExecutionBlock((ExecutionBlockStopEvent) event); + stopExecutionBlock(executionBlockContextMap.remove(event.getExecutionBlockId()), + ((ExecutionBlockStopEvent) event).getCleanupList()); } } diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeNodeResourceManager.java b/tajo-core/src/test/java/org/apache/tajo/worker/MockExecutionBlock.java similarity index 54% rename from tajo-core/src/test/java/org/apache/tajo/worker/MockNodeNodeResourceManager.java rename to tajo-core/src/test/java/org/apache/tajo/worker/MockExecutionBlock.java index 76e32163e6..9d4e1f3392 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeNodeResourceManager.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/MockExecutionBlock.java @@ -18,37 +18,25 @@ package org.apache.tajo.worker; -import org.apache.hadoop.yarn.event.Dispatcher; -import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.tajo.TaskAttemptId; import org.apache.tajo.ipc.TajoWorkerProtocol; -import org.apache.tajo.resource.NodeResource; -public class MockNodeNodeResourceManager extends NodeResourceManager { +import java.io.IOException; - volatile boolean enableTaskHandlerEvent; +public class MockExecutionBlock extends ExecutionBlockContext { - public MockNodeNodeResourceManager(Dispatcher dispatcher, EventHandler taskEventHandler) { - super(dispatcher, taskEventHandler); + public MockExecutionBlock(TajoWorker.WorkerContext workerContext, + TajoWorkerProtocol.RunExecutionBlockRequestProto request) throws IOException { + super(workerContext, null, request); } @Override - protected void startExecutionBlock(TajoWorkerProtocol.RunExecutionBlockRequestProto request) { - if(enableTaskHandlerEvent) { - super.startExecutionBlock(request); - } + public void init() throws Throwable { + //skip } @Override - protected void startTask(TajoWorkerProtocol.TaskRequestProto request, NodeResource resource) { - if(enableTaskHandlerEvent) { - super.startTask(request, resource); - } - } + public void fatalError(TaskAttemptId taskAttemptId, String message) { - /** - * skip task execution and deallocation for testing - * */ - public void setTaskHandlerEvent(boolean flag) { - enableTaskHandlerEvent = flag; } } diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeResourceManager.java b/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeResourceManager.java new file mode 100644 index 0000000000..18b9405c8a --- /dev/null +++ b/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeResourceManager.java @@ -0,0 +1,96 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.worker; + +import org.apache.hadoop.yarn.event.Dispatcher; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.tajo.ExecutionBlockId; +import org.apache.tajo.QueryIdFactory; +import org.apache.tajo.TaskAttemptId; +import org.apache.tajo.ipc.TajoWorkerProtocol; +import org.apache.tajo.plan.serder.PlanProto; +import org.apache.tajo.resource.NodeResource; +import org.apache.tajo.resource.NodeResources; +import org.apache.tajo.worker.event.NodeResourceEvent; + +import java.util.Queue; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.Semaphore; + +public class MockNodeResourceManager extends NodeResourceManager { + + volatile boolean enableTaskHandlerEvent = true; + private final Semaphore barrier; + + public MockNodeResourceManager(Semaphore barrier, Dispatcher dispatcher, EventHandler taskEventHandler) { + super(dispatcher, taskEventHandler); + this.barrier = barrier; + } + + @Override + public void handle(NodeResourceEvent event) { + super.handle(event); + barrier.release(); + } + + @Override + protected void startExecutionBlock(TajoWorkerProtocol.RunExecutionBlockRequestProto request) { + if(enableTaskHandlerEvent) { + super.startExecutionBlock(request); + } + } + + @Override + protected void startTask(TajoWorkerProtocol.TaskRequestProto request, NodeResource resource) { + if(enableTaskHandlerEvent) { + super.startTask(request, resource); + } + } + + /** + * skip task execution and deallocation for testing + * */ + public void setTaskHandlerEvent(boolean flag) { + enableTaskHandlerEvent = flag; + } + + protected static Queue createTaskRequests( + ExecutionBlockId ebId, int memory, int size) { + + Queue + requestProtoList = new LinkedBlockingQueue(); + for (int i = 0; i < size; i++) { + + TaskAttemptId taskAttemptId = QueryIdFactory.newTaskAttemptId(QueryIdFactory.newTaskId(ebId, i), 0); + TajoWorkerProtocol.TaskRequestProto.Builder builder = + TajoWorkerProtocol.TaskRequestProto.newBuilder(); + builder.setId(taskAttemptId.getProto()); + builder.setShouldDie(true); + builder.setOutputTable(""); + builder.setPlan(PlanProto.LogicalNodeTree.newBuilder()); + builder.setClusteredOutput(false); + + + requestProtoList.add(TajoWorkerProtocol.TaskAllocationRequestProto.newBuilder() + .setResource(NodeResources.createResource(memory).getProto()) + .setTaskRequest(builder.build()).build()); + } + return requestProtoList; + } +} diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskExecutor.java b/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskExecutor.java index 1748546b6b..f62733f1a2 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskExecutor.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskExecutor.java @@ -23,18 +23,31 @@ import org.apache.tajo.TaskAttemptId; import org.apache.tajo.catalog.statistics.TableStats; import org.apache.tajo.ipc.TajoWorkerProtocol; +import org.apache.tajo.worker.event.TaskExecutorEvent; import java.io.IOException; +import java.util.concurrent.Semaphore; public class MockTaskExecutor extends TaskExecutor { - public MockTaskExecutor(TaskManager taskManager, EventHandler rmEventHandler) { + protected final Semaphore barrier; + + public MockTaskExecutor(Semaphore barrier, TaskManager taskManager, EventHandler rmEventHandler) { super(taskManager, rmEventHandler); + this.barrier = barrier; + } + + @Override + public void handle(TaskExecutorEvent event) { + super.handle(event); + barrier.release(); } @Override protected Task createTask(final ExecutionBlockContext context, TajoWorkerProtocol.TaskRequestProto taskRequest) { final TaskAttemptId taskAttemptId = new TaskAttemptId(taskRequest.getId()); + + //ignore status changed log final TaskAttemptContext taskAttemptContext = new TaskAttemptContext(null, context, taskAttemptId, null, null) { private TajoProtos.TaskAttemptState state; diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskManager.java b/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskManager.java new file mode 100644 index 0000000000..678b0636c4 --- /dev/null +++ b/tajo-core/src/test/java/org/apache/tajo/worker/MockTaskManager.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.tajo.worker; + +import org.apache.hadoop.yarn.event.Dispatcher; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.tajo.ipc.TajoWorkerProtocol; +import org.apache.tajo.resource.NodeResource; +import org.apache.tajo.worker.event.TaskManagerEvent; + +import java.io.IOException; +import java.util.concurrent.Semaphore; + +public class MockTaskManager extends TaskManager { + + private final Semaphore barrier; + + public MockTaskManager(Semaphore barrier, Dispatcher dispatcher, TajoWorker.WorkerContext workerContext, EventHandler rmEventHandler) { + super(dispatcher, workerContext, rmEventHandler); + this.barrier = barrier; + } + + @Override + protected ExecutionBlockContext createExecutionBlock(TajoWorkerProtocol.RunExecutionBlockRequestProto request) { + try { + return new MockExecutionBlock(getWorkerContext(), request); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + protected void stopExecutionBlock(ExecutionBlockContext context, + TajoWorkerProtocol.ExecutionBlockListProto cleanupList) { + //skip for testing + } + + @Override + public void handle(TaskManagerEvent event) { + super.handle(event); + barrier.release(); + } +} diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/MockWorkerContext.java b/tajo-core/src/test/java/org/apache/tajo/worker/MockWorkerContext.java new file mode 100644 index 0000000000..e8c2b9c369 --- /dev/null +++ b/tajo-core/src/test/java/org/apache/tajo/worker/MockWorkerContext.java @@ -0,0 +1,129 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.worker; + +import org.apache.hadoop.fs.LocalDirAllocator; +import org.apache.tajo.catalog.CatalogService; +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.ipc.QueryCoordinatorProtocol; +import org.apache.tajo.master.cluster.WorkerConnectionInfo; +import org.apache.tajo.querymaster.QueryMaster; +import org.apache.tajo.querymaster.QueryMasterManagerService; +import org.apache.tajo.service.ServiceTracker; +import org.apache.tajo.storage.HashShuffleAppenderManager; +import org.apache.tajo.util.history.HistoryReader; +import org.apache.tajo.util.history.HistoryWriter; +import org.apache.tajo.util.metrics.TajoSystemMetrics; + +public abstract class MockWorkerContext implements TajoWorker.WorkerContext { + TajoSystemMetrics tajoSystemMetrics; + + @Override + public QueryMaster getQueryMaster() { + return null; + } + + public abstract TajoConf getConf(); + + @Override + public ServiceTracker getServiceTracker() { + return null; + } + + @Override + public QueryMasterManagerService getQueryMasterManagerService() { + return null; + } + + @Override + public TaskRunnerManager getTaskRunnerManager() { + return null; + } + + @Override + public CatalogService getCatalog() { + return null; + } + + @Override + public WorkerConnectionInfo getConnectionInfo() { + return null; + } + + @Override + public String getWorkerName() { + return null; + } + + @Override + public LocalDirAllocator getLocalDirAllocator() { + return null; + } + + @Override + public QueryCoordinatorProtocol.ClusterResourceSummary getClusterResource() { + return null; + } + + @Override + public TajoSystemMetrics getWorkerSystemMetrics() { + + if (tajoSystemMetrics == null) { + tajoSystemMetrics = new TajoSystemMetrics(getConf(), "test-file-group", "localhost"); + tajoSystemMetrics.start(); + } + return tajoSystemMetrics; + } + + @Override + public HashShuffleAppenderManager getHashShuffleAppenderManager() { + return null; + } + + @Override + public HistoryWriter getTaskHistoryWriter() { + return null; + } + + @Override + public HistoryReader getHistoryReader() { + return null; + } + + @Override + public void cleanup(String strPath) { + + } + + @Override + public void cleanupTemporalDirectories() { + + } + + @Override + public void setClusterResource(QueryCoordinatorProtocol.ClusterResourceSummary clusterResource) { + + } + + @Override + public void setNumClusterNodes(int numClusterNodes) { + + } +} + diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java index 0b088b031d..68111b8aa6 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java @@ -24,10 +24,10 @@ import org.apache.hadoop.yarn.event.AsyncDispatcher; import org.apache.tajo.*; import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.ipc.TajoWorkerProtocol; import org.apache.tajo.master.cluster.WorkerConnectionInfo; import org.apache.tajo.plan.serder.PlanProto; -import org.apache.tajo.resource.NodeResources; import org.apache.tajo.rpc.CallFuture; import org.apache.tajo.util.CommonTestingUtil; import org.apache.tajo.worker.event.NodeResourceAllocateEvent; @@ -44,7 +44,7 @@ import static org.apache.tajo.ipc.TajoWorkerProtocol.*; public class TestNodeResourceManager { - private MockNodeNodeResourceManager resourceManager; + private MockNodeResourceManager resourceManager; private NodeStatusUpdater statusUpdater; private TaskManager taskManager; private TaskExecutor taskExecutor; @@ -54,6 +54,7 @@ public class TestNodeResourceManager { private CompositeService service; private int taskMemory; private TajoConf conf; + private WorkerConnectionInfo worker; @Before public void setup() { @@ -70,10 +71,18 @@ public void setup() { dispatcher = new AsyncDispatcher(); taskDispatcher = new AsyncDispatcher(); - taskManager = new TaskManager(taskDispatcher, null, dispatcher.getEventHandler()); - taskExecutor = new MockTaskExecutor(taskManager, dispatcher.getEventHandler()); - resourceManager = new MockNodeNodeResourceManager(dispatcher, taskDispatcher.getEventHandler()); - WorkerConnectionInfo worker = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); + final TajoWorker.WorkerContext workerContext = new MockWorkerContext() { + + @Override + public TajoConf getConf() { + return conf; + } + }; + + taskManager = new MockTaskManager(new Semaphore(0), taskDispatcher, workerContext, dispatcher.getEventHandler()); + taskExecutor = new MockTaskExecutor(new Semaphore(0), taskManager, dispatcher.getEventHandler()); + resourceManager = new MockNodeResourceManager(new Semaphore(0), dispatcher, taskDispatcher.getEventHandler()); + worker = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); statusUpdater = new MockNodeStatusUpdater(new CountDownLatch(0), worker, resourceManager); service = new CompositeService("MockService") { @@ -109,7 +118,7 @@ public void testNodeResourceAllocateEvent() throws Exception { requestProto.setExecutionBlockId(ebId.getProto()); assertEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); - requestProto.addAllTaskRequest(createTaskRequests(taskMemory, requestSize)); + requestProto.addAllTaskRequest(MockNodeResourceManager.createTaskRequests(ebId, taskMemory, requestSize)); dispatcher.getEventHandler().handle(new NodeResourceAllocateEvent(requestProto.build(), callFuture)); @@ -132,7 +141,8 @@ public void testNodeResourceCancellation() throws Exception { requestProto.setExecutionBlockId(ebId.getProto()); assertEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); - requestProto.addAllTaskRequest(createTaskRequests(taskMemory, requestSize + overSize)); + requestProto.addAllTaskRequest( + MockNodeResourceManager.createTaskRequests(ebId, taskMemory, requestSize + overSize)); dispatcher.getEventHandler().handle(new NodeResourceAllocateEvent(requestProto.build(), callFuture)); BatchAllocationResponseProto responseProto = callFuture.get(); @@ -151,7 +161,7 @@ public void testNodeResourceDeallocateEvent() throws Exception { requestProto.setExecutionBlockId(ebId.getProto()); assertEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); - requestProto.addAllTaskRequest(createTaskRequests(taskMemory, requestSize)); + requestProto.addAllTaskRequest(MockNodeResourceManager.createTaskRequests(ebId, taskMemory, requestSize)); dispatcher.getEventHandler().handle(new NodeResourceAllocateEvent(requestProto.build(), callFuture)); @@ -178,8 +188,31 @@ public void testParallelRequest() throws Exception { final AtomicInteger totalCanceled = new AtomicInteger(); final ExecutionBlockId ebId = new ExecutionBlockId(LocalTajoTestingUtility.newQueryId(), 0); - final Queue totalTasks = createTaskRequests(taskMemory, taskSize); + final Queue + totalTasks = MockNodeResourceManager.createTaskRequests(ebId, taskMemory, taskSize); + + // first request with starting ExecutionBlock + TajoWorkerProtocol.RunExecutionBlockRequestProto.Builder + ebRequestProto = TajoWorkerProtocol.RunExecutionBlockRequestProto.newBuilder(); + ebRequestProto.setExecutionBlockId(ebId.getProto()) + .setQueryMaster(worker.getProto()) + .setNodeId(worker.getHost()+":" + worker.getQueryMasterPort()) + .setContainerId("test") + .setQueryContext(new QueryContext(conf).getProto()) + .setPlanJson("test") + .setShuffleType(PlanProto.ShuffleType.HASH_SHUFFLE); + + TaskAllocationRequestProto task = totalTasks.poll(); + BatchAllocationRequestProto.Builder requestProto = BatchAllocationRequestProto.newBuilder(); + requestProto.addTaskRequest(task); + requestProto.setExecutionBlockId(ebId.getProto()); + requestProto.setExecutionBlockRequest(ebRequestProto.build()); + CallFuture callFuture = new CallFuture(); + dispatcher.getEventHandler().handle(new NodeResourceAllocateEvent(requestProto.build(), callFuture)); + assertTrue(callFuture.get().getCancellationTaskCount() == 0); + totalComplete.incrementAndGet(); + // start parallel request ExecutorService executor = Executors.newFixedThreadPool(parallelCount); List futureList = Lists.newArrayList(); @@ -228,26 +261,4 @@ public void run() { executor.shutdown(); assertEquals(taskSize, totalComplete.get()); } - - protected static Queue createTaskRequests(int memory, int size) { - Queue requestProtoList = new LinkedBlockingQueue(); - for (int i = 0; i < size; i++) { - - ExecutionBlockId nullStage = QueryIdFactory.newExecutionBlockId(QueryIdFactory.NULL_QUERY_ID, 0); - TaskAttemptId taskAttemptId = QueryIdFactory.newTaskAttemptId(QueryIdFactory.newTaskId(nullStage, i), 0); - TajoWorkerProtocol.TaskRequestProto.Builder builder = - TajoWorkerProtocol.TaskRequestProto.newBuilder(); - builder.setId(taskAttemptId.getProto()); - builder.setShouldDie(true); - builder.setOutputTable(""); - builder.setPlan(PlanProto.LogicalNodeTree.newBuilder()); - builder.setClusteredOutput(false); - - - requestProtoList.add(TaskAllocationRequestProto.newBuilder() - .setResource(NodeResources.createResource(memory).getProto()) - .setTaskRequest(builder.build()).build()); - } - return requestProtoList; - } } diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java index 0139088dad..071a95d530 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java @@ -22,6 +22,7 @@ import org.apache.tajo.conf.TajoConf; import org.apache.tajo.ipc.TajoResourceTrackerProtocol; import org.apache.tajo.master.cluster.WorkerConnectionInfo; +import org.apache.tajo.master.rm.Worker; import org.apache.tajo.util.CommonTestingUtil; import org.apache.tajo.worker.event.NodeStatusEvent; import org.junit.After; @@ -37,11 +38,14 @@ public class TestNodeStatusUpdater { private MockNodeStatusUpdater statusUpdater; private AsyncDispatcher dispatcher; private TajoConf conf; + private WorkerConnectionInfo worker; + @Before public void setup() { conf = new TajoConf(); conf.set(CommonTestingUtil.TAJO_TEST_KEY, CommonTestingUtil.TAJO_TEST_TRUE); + worker = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); conf.setIntVar(TajoConf.ConfVars.WORKER_HEARTBEAT_INTERVAL, 1000); dispatcher = new AsyncDispatcher(); @@ -63,7 +67,6 @@ public void tearDown() { @Test(timeout = 20000) public void testNodeMembership() throws Exception { CountDownLatch barrier = new CountDownLatch(1); - WorkerConnectionInfo worker = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); statusUpdater = new MockNodeStatusUpdater(barrier, worker, resourceManager); statusUpdater.init(conf); statusUpdater.start(); @@ -82,7 +85,6 @@ public void testNodeMembership() throws Exception { @Test(timeout = 20000) public void testPing() throws Exception { CountDownLatch barrier = new CountDownLatch(2); - WorkerConnectionInfo worker = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); statusUpdater = new MockNodeStatusUpdater(barrier, worker, resourceManager); statusUpdater.init(conf); statusUpdater.start(); @@ -100,7 +102,6 @@ public void testPing() throws Exception { @Test(timeout = 20000) public void testResourceReport() throws Exception { CountDownLatch barrier = new CountDownLatch(2); - WorkerConnectionInfo worker = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); statusUpdater = new MockNodeStatusUpdater(barrier, worker, resourceManager); statusUpdater.init(conf); statusUpdater.start(); @@ -116,7 +117,6 @@ public void testResourceReport() throws Exception { @Test(timeout = 20000) public void testFlushResourceReport() throws Exception { CountDownLatch barrier = new CountDownLatch(2); - WorkerConnectionInfo worker = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); statusUpdater = new MockNodeStatusUpdater(barrier, worker, resourceManager); statusUpdater.init(conf); statusUpdater.start(); diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java new file mode 100644 index 0000000000..7e240e5ddf --- /dev/null +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java @@ -0,0 +1,320 @@ +/** + * 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.tajo.worker; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.LocalDirAllocator; +import org.apache.hadoop.service.CompositeService; +import org.apache.hadoop.yarn.event.AsyncDispatcher; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.tajo.*; +import org.apache.tajo.annotation.ThreadSafe; +import org.apache.tajo.catalog.CatalogService; +import org.apache.tajo.catalog.statistics.TableStats; +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.engine.query.QueryContext; +import org.apache.tajo.ipc.QueryCoordinatorProtocol; +import org.apache.tajo.ipc.TajoWorkerProtocol; +import org.apache.tajo.master.cluster.WorkerConnectionInfo; +import org.apache.tajo.plan.serder.PlanProto; +import org.apache.tajo.querymaster.QueryMaster; +import org.apache.tajo.querymaster.QueryMasterManagerService; +import org.apache.tajo.resource.NodeResources; +import org.apache.tajo.rpc.CallFuture; +import org.apache.tajo.service.ServiceTracker; +import org.apache.tajo.storage.HashShuffleAppenderManager; +import org.apache.tajo.util.CommonTestingUtil; +import org.apache.tajo.util.history.HistoryReader; +import org.apache.tajo.util.history.HistoryWriter; +import org.apache.tajo.util.metrics.TajoSystemMetrics; +import org.apache.tajo.worker.event.ExecutionBlockStartEvent; +import org.apache.tajo.worker.event.ExecutionBlockStopEvent; +import org.apache.tajo.worker.event.NodeResourceAllocateEvent; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.Queue; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.apache.tajo.ipc.TajoWorkerProtocol.*; +import static org.junit.Assert.*; + +public class TestTaskExecutor { + + private NodeResourceManager resourceManager; + private NodeStatusUpdater statusUpdater; + private TaskManager taskManager; + private TaskExecutor taskExecutor; + private AsyncDispatcher dispatcher; + private AsyncDispatcher taskDispatcher; + private WorkerConnectionInfo worker; + + private CompositeService service; + private TajoConf conf; + private Semaphore barrier; + private Semaphore resourceManagerBarrier; + + @Before + public void setup() { + conf = new TajoConf(); + conf.set(CommonTestingUtil.TAJO_TEST_KEY, CommonTestingUtil.TAJO_TEST_TRUE); + dispatcher = new AsyncDispatcher(); + taskDispatcher = new AsyncDispatcher(); + + final TajoWorker.WorkerContext workerContext = new MockWorkerContext() { + + @Override + public TajoConf getConf() { + return conf; + } + }; + + barrier = new Semaphore(0); + resourceManagerBarrier = new Semaphore(0); + taskManager = new MockTaskManager(new Semaphore(0), taskDispatcher, workerContext, dispatcher.getEventHandler()); + taskExecutor = new TaskExecutor(barrier, taskManager, dispatcher.getEventHandler()); + resourceManager = new MockNodeResourceManager(resourceManagerBarrier, dispatcher, taskDispatcher.getEventHandler()); + worker = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); + statusUpdater = new MockNodeStatusUpdater(new CountDownLatch(0), worker, resourceManager); + + service = new CompositeService("MockService") { + @Override + protected void serviceInit(Configuration conf) throws Exception { + addIfService(dispatcher); + addIfService(taskDispatcher); + addIfService(taskManager); + addIfService(taskExecutor); + addIfService(resourceManager); + addIfService(statusUpdater); + super.serviceInit(conf); + } + + + @Override + protected void serviceStop() throws Exception { + workerContext.getWorkerSystemMetrics().stop(); + super.serviceStop(); + } + }; + + service.init(conf); + service.start(); + } + + @After + public void tearDown() { + service.stop(); + } + + @Test + public void testTaskRequest() throws Exception { + int requestSize = 1; + + RunExecutionBlockRequestProto.Builder + ebRequestProto = RunExecutionBlockRequestProto.newBuilder(); + QueryId qid = LocalTajoTestingUtility.newQueryId(); + ExecutionBlockId ebId = QueryIdFactory.newExecutionBlockId(qid, 1); + + ebRequestProto.setExecutionBlockId(ebId.getProto()) + .setQueryMaster(worker.getProto()) + .setNodeId(worker.getHost()+":" + worker.getQueryMasterPort()) + .setContainerId("test") + .setQueryContext(new QueryContext(conf).getProto()) + .setPlanJson("test") + .setShuffleType(PlanProto.ShuffleType.HASH_SHUFFLE); + + CallFuture callFuture = new CallFuture(); + BatchAllocationRequestProto.Builder requestProto = BatchAllocationRequestProto.newBuilder(); + requestProto.setExecutionBlockId(ebId.getProto()); + requestProto.setExecutionBlockRequest(ebRequestProto.build()); + + assertEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); + requestProto.addAllTaskRequest(MockNodeResourceManager.createTaskRequests(ebId, 10, requestSize)); + + dispatcher.getEventHandler().handle(new NodeResourceAllocateEvent(requestProto.build(), callFuture)); + + //verify running task + assertTrue(barrier.tryAcquire(3, TimeUnit.SECONDS)); + assertEquals(1, taskExecutor.getRunningTasks()); + assertTrue(barrier.tryAcquire(3, TimeUnit.SECONDS)); + assertEquals(0, taskExecutor.getRunningTasks()); + assertEquals(1, taskExecutor.completeTasks); + + //verify the released resources + Thread.sleep(100); + assertEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); + } + + @Test + public void testTaskException() throws Exception { + int requestSize = 1; + + RunExecutionBlockRequestProto.Builder + ebRequestProto = RunExecutionBlockRequestProto.newBuilder(); + QueryId qid = LocalTajoTestingUtility.newQueryId(); + ExecutionBlockId ebId = QueryIdFactory.newExecutionBlockId(qid, 1); + + ebRequestProto.setExecutionBlockId(ebId.getProto()) + .setQueryMaster(worker.getProto()) + .setNodeId(worker.getHost()+":" + worker.getQueryMasterPort()) + .setContainerId("test") + .setQueryContext(new QueryContext(conf).getProto()) + .setPlanJson("test") + .setShuffleType(PlanProto.ShuffleType.HASH_SHUFFLE); + + CallFuture callFuture = new CallFuture(); + BatchAllocationRequestProto.Builder requestProto = BatchAllocationRequestProto.newBuilder(); + requestProto.setExecutionBlockId(ebId.getProto()); + requestProto.setExecutionBlockRequest(ebRequestProto.build()); + + assertEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); + requestProto.addAllTaskRequest(MockNodeResourceManager.createTaskRequests(ebId, 10, requestSize)); + + taskExecutor.throwException.set(true); + dispatcher.getEventHandler().handle(new NodeResourceAllocateEvent(requestProto.build(), callFuture)); + + //verify running task + assertTrue(barrier.tryAcquire(3, TimeUnit.SECONDS)); + assertEquals(1, taskExecutor.getRunningTasks()); + assertTrue(barrier.tryAcquire(3, TimeUnit.SECONDS)); + assertEquals(0, taskExecutor.getRunningTasks()); + assertEquals(0, taskExecutor.completeTasks); + + //verify the released resources + Thread.sleep(100); + assertEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); + } + + class TaskExecutor extends MockTaskExecutor { + int completeTasks; + AtomicBoolean throwException = new AtomicBoolean(); + + public TaskExecutor(Semaphore barrier, TaskManager taskManager, EventHandler rmEventHandler) { + super(barrier, taskManager, rmEventHandler); + } + + @Override + protected void stopTask(TaskAttemptId taskId) { + super.stopTask(taskId); + super.barrier.release(); + } + + @Override + protected Task createTask(final ExecutionBlockContext context, TajoWorkerProtocol.TaskRequestProto taskRequest) { + final TaskAttemptId taskAttemptId = new TaskAttemptId(taskRequest.getId()); + final TaskAttemptContext taskAttemptContext = new TaskAttemptContext(null, context, taskAttemptId, null, null); + + return new Task() { + @Override + public void init() throws IOException { + + try { + Thread.sleep(50); + } catch (InterruptedException e) { + e.printStackTrace(); + } + } + + @Override + public void fetch() { + try { + Thread.sleep(50); + } catch (InterruptedException e) { + e.printStackTrace(); + } + } + + @Override + public void run() throws Exception { + Thread.sleep(50); + + if(throwException.get()) throw new RuntimeException(); + + taskAttemptContext.stop(); + taskAttemptContext.setProgress(1.0f); + taskAttemptContext.setState(TajoProtos.TaskAttemptState.TA_SUCCEEDED); + completeTasks++; + } + + @Override + public void kill() { + + } + + @Override + public void abort() { + + } + + @Override + public void cleanup() { + } + + @Override + public boolean hasFetchPhase() { + return false; + } + + @Override + public boolean isProgressChanged() { + return false; + } + + @Override + public boolean isStopped() { + return taskAttemptContext.isStopped(); + } + + @Override + public void updateProgress() { + + } + + @Override + public TaskAttemptContext getTaskContext() { + return taskAttemptContext; + } + + @Override + public ExecutionBlockContext getExecutionBlockContext() { + return context; + } + + @Override + public TajoWorkerProtocol.TaskStatusProto getReport() { + TajoWorkerProtocol.TaskStatusProto.Builder builder = TajoWorkerProtocol.TaskStatusProto.newBuilder(); + builder.setWorkerName("localhost:0"); + builder.setId(taskAttemptContext.getTaskId().getProto()) + .setProgress(taskAttemptContext.getProgress()) + .setState(taskAttemptContext.getState()); + + builder.setInputStats(new TableStats().getProto()); + return builder.build(); + } + }; + } + } +} diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java new file mode 100644 index 0000000000..6cdcf14e31 --- /dev/null +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java @@ -0,0 +1,175 @@ +/** + * 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.tajo.worker; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.service.CompositeService; +import org.apache.hadoop.yarn.event.AsyncDispatcher; +import org.apache.tajo.*; +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.engine.query.QueryContext; +import org.apache.tajo.ipc.TajoWorkerProtocol; +import org.apache.tajo.master.cluster.WorkerConnectionInfo; +import org.apache.tajo.plan.serder.PlanProto; +import org.apache.tajo.rpc.CallFuture; +import org.apache.tajo.util.CommonTestingUtil; +import org.apache.tajo.worker.event.ExecutionBlockStartEvent; +import org.apache.tajo.worker.event.ExecutionBlockStopEvent; +import org.apache.tajo.worker.event.NodeResourceAllocateEvent; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.util.concurrent.*; + +import static org.apache.tajo.ipc.TajoWorkerProtocol.*; +import static org.junit.Assert.*; + +public class TestTaskManager { + + private NodeResourceManager resourceManager; + private NodeStatusUpdater statusUpdater; + private TaskManager taskManager; + private TaskExecutor taskExecutor; + private AsyncDispatcher dispatcher; + private AsyncDispatcher taskDispatcher; + private WorkerConnectionInfo worker; + + private CompositeService service; + private int taskMemory; + private TajoConf conf; + private Semaphore barrier; + + @Before + public void setup() { + conf = new TajoConf(); + conf.set(CommonTestingUtil.TAJO_TEST_KEY, CommonTestingUtil.TAJO_TEST_TRUE); + + taskMemory = 512; + conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES, 4); + conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_MEMORY_MB, + taskMemory * conf.getIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_CPU_CORES)); + conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_DISKS_NUM, 4); + conf.setIntVar(TajoConf.ConfVars.WORKER_RESOURCE_AVAILABLE_DISK_PARALLEL_NUM, 1); + + dispatcher = new AsyncDispatcher(); + taskDispatcher = new AsyncDispatcher(); + + final TajoWorker.WorkerContext workerContext = new MockWorkerContext() { + + @Override + public TajoConf getConf() { + return conf; + } + }; + barrier = new Semaphore(0); + taskManager = new MockTaskManager(barrier, taskDispatcher, workerContext, dispatcher.getEventHandler()); + taskExecutor = new MockTaskExecutor(new Semaphore(0), taskManager, dispatcher.getEventHandler()); + resourceManager = new NodeResourceManager(dispatcher, taskDispatcher.getEventHandler()); + worker = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); + statusUpdater = new MockNodeStatusUpdater(new CountDownLatch(0), worker, resourceManager); + + service = new CompositeService("MockService") { + @Override + protected void serviceInit(Configuration conf) throws Exception { + addIfService(dispatcher); + addIfService(taskDispatcher); + addIfService(taskManager); + addIfService(taskExecutor); + addIfService(resourceManager); + addIfService(statusUpdater); + super.serviceInit(conf); + } + + + @Override + protected void serviceStop() throws Exception { + workerContext.getWorkerSystemMetrics().stop(); + super.serviceStop(); + } + }; + + service.init(conf); + service.start(); + } + + @After + public void tearDown() { + service.stop(); + } + + @Test(timeout = 10000) + public void testExecutionBlockStart() throws Exception { + int requestSize = 1; + + TajoWorkerProtocol.RunExecutionBlockRequestProto.Builder + ebRequestProto = TajoWorkerProtocol.RunExecutionBlockRequestProto.newBuilder(); + QueryId qid = LocalTajoTestingUtility.newQueryId(); + ExecutionBlockId ebId = QueryIdFactory.newExecutionBlockId(qid, 1); + + ebRequestProto.setExecutionBlockId(ebId.getProto()) + .setQueryMaster(worker.getProto()) + .setNodeId(worker.getHost()+":" + worker.getQueryMasterPort()) + .setContainerId("test") + .setQueryContext(new QueryContext(conf).getProto()) + .setPlanJson("test") + .setShuffleType(PlanProto.ShuffleType.HASH_SHUFFLE); + + CallFuture callFuture = new CallFuture(); + BatchAllocationRequestProto.Builder requestProto = BatchAllocationRequestProto.newBuilder(); + requestProto.setExecutionBlockId(ebId.getProto()); + requestProto.setExecutionBlockRequest(ebRequestProto.build()); + + assertEquals(resourceManager.getTotalResource(), resourceManager.getAvailableResource()); + requestProto.addAllTaskRequest(MockNodeResourceManager.createTaskRequests(ebId, taskMemory, requestSize)); + + dispatcher.getEventHandler().handle(new NodeResourceAllocateEvent(requestProto.build(), callFuture)); + + assertTrue(barrier.tryAcquire(3, TimeUnit.SECONDS)); + assertNotNull(taskManager.getExecutionBlockContext(ebId)); + assertEquals(ebId, taskManager.getExecutionBlockContext(ebId).getExecutionBlockId()); + } + + @Test(timeout = 10000) + public void testExecutionBlockStop() throws Exception { + + TajoWorkerProtocol.RunExecutionBlockRequestProto.Builder + ebRequestProto = TajoWorkerProtocol.RunExecutionBlockRequestProto.newBuilder(); + QueryId qid = LocalTajoTestingUtility.newQueryId(); + ExecutionBlockId ebId = QueryIdFactory.newExecutionBlockId(qid, 1); + + ebRequestProto.setExecutionBlockId(ebId.getProto()) + .setQueryMaster(worker.getProto()) + .setNodeId(worker.getHost()+":" + worker.getQueryMasterPort()) + .setContainerId("test") + .setQueryContext(new QueryContext(conf).getProto()) + .setPlanJson("test") + .setShuffleType(PlanProto.ShuffleType.HASH_SHUFFLE); + + taskDispatcher.getEventHandler().handle(new ExecutionBlockStartEvent(ebRequestProto.build())); + assertTrue(barrier.tryAcquire(3, TimeUnit.SECONDS)); + assertNotNull(taskManager.getExecutionBlockContext(ebId)); + assertEquals(ebId, taskManager.getExecutionBlockContext(ebId).getExecutionBlockId()); + + ExecutionBlockListProto.Builder ebList = ExecutionBlockListProto.newBuilder(); + taskDispatcher.getEventHandler().handle(new ExecutionBlockStopEvent(ebId.getProto(), ebList.build())); + assertTrue(barrier.tryAcquire(3, TimeUnit.SECONDS)); + assertNull(taskManager.getExecutionBlockContext(ebId)); + } +} From 1ca09fda5a97747a79f78af9b9bc66c92a625f76 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Mon, 1 Jun 2015 10:34:46 +0900 Subject: [PATCH 18/21] TAJO-1615 --- .../tajo/master/rm/TajoResourceTracker.java | 4 ++- .../tajo/util/history/HistoryWriter.java | 2 +- .../apache/tajo/worker/NodeStatusUpdater.java | 17 +++++----- .../tajo/worker/MockNodeStatusUpdater.java | 4 +-- .../tajo/worker/TestNodeResourceManager.java | 22 +++++++++---- .../tajo/worker/TestNodeStatusUpdater.java | 33 ++++++++++++++----- .../apache/tajo/worker/TestTaskExecutor.java | 26 ++++++++++----- .../apache/tajo/worker/TestTaskManager.java | 26 ++++++++++----- 8 files changed, 90 insertions(+), 44 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java index af28886446..2a18de7ecb 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoResourceTracker.java @@ -188,7 +188,9 @@ public void heartbeat( public void nodeHeartbeat(RpcController controller, TajoResourceTrackerProtocol.NodeHeartbeatRequestProto request, RpcCallback done) { //TODO implement with ResourceManager for scheduler - throw new RuntimeException(new ServiceException(new NotImplementedException().getMessage())); + TajoResourceTrackerProtocol.NodeHeartbeatResponseProto.Builder + response = TajoResourceTrackerProtocol.NodeHeartbeatResponseProto.newBuilder(); + done.run(response.setCommand(TajoResourceTrackerProtocol.ResponseCommand.NORMAL).build()); } private Worker createWorkerResource(NodeHeartbeat request) { diff --git a/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryWriter.java b/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryWriter.java index e8ba3046b9..daced3e90c 100644 --- a/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryWriter.java +++ b/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryWriter.java @@ -217,7 +217,7 @@ class WriterThread extends Thread { public void run() { LOG.info("HistoryWriter_" + processName + " started."); SimpleDateFormat df = new SimpleDateFormat("yyyyMMddHH"); - while (!stopped.get()) { + while (!stopped.get() && !Thread.interrupted()) { List> histories = Lists.newArrayList(); try { diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java b/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java index 6bf4a67879..d13cd50a32 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/NodeStatusUpdater.java @@ -57,16 +57,16 @@ public class NodeStatusUpdater extends AbstractService implements EventHandler heartBeatRequestQueue; - private final WorkerConnectionInfo connectionInfo; + private final TajoWorker.WorkerContext workerContext; private final NodeResourceManager nodeResourceManager; private AsyncRpcClient rmClient; private ServiceTracker serviceTracker; private TajoResourceTrackerProtocolService.Interface resourceTracker; private int queueingLimit; - public NodeStatusUpdater(WorkerConnectionInfo connectionInfo, NodeResourceManager resourceManager) { + public NodeStatusUpdater(TajoWorker.WorkerContext workerContext, NodeResourceManager resourceManager) { super(NodeStatusUpdater.class.getSimpleName()); - this.connectionInfo = connectionInfo; + this.workerContext = workerContext; this.nodeResourceManager = resourceManager; } @@ -99,7 +99,8 @@ public void serviceStop() throws Exception { this.isStopped = true; synchronized (updaterThread) { - updaterThread.notifyAll(); + updaterThread.interrupt(); + updaterThread.join(); } super.serviceStop(); LOG.info("NodeStatusUpdater stopped."); @@ -121,13 +122,13 @@ public int getQueueingLimit() { private NodeHeartbeatRequestProto createResourceReport(NodeResource resource) { NodeHeartbeatRequestProto.Builder requestProto = NodeHeartbeatRequestProto.newBuilder(); requestProto.setAvailableResource(resource.getProto()); - requestProto.setWorkerId(connectionInfo.getId()); + requestProto.setWorkerId(workerContext.getConnectionInfo().getId()); return requestProto.build(); } private NodeHeartbeatRequestProto createHeartBeatReport() { NodeHeartbeatRequestProto.Builder requestProto = NodeHeartbeatRequestProto.newBuilder(); - requestProto.setWorkerId(connectionInfo.getId()); + requestProto.setWorkerId(workerContext.getConnectionInfo().getId()); return requestProto.build(); } @@ -135,8 +136,8 @@ private NodeHeartbeatRequestProto createNodeStatusReport() { NodeHeartbeatRequestProto.Builder requestProto = NodeHeartbeatRequestProto.newBuilder(); requestProto.setTotalResource(nodeResourceManager.getTotalResource().getProto()); requestProto.setAvailableResource(nodeResourceManager.getAvailableResource().getProto()); - requestProto.setWorkerId(connectionInfo.getId()); - requestProto.setConnectionInfo(connectionInfo.getProto()); + requestProto.setWorkerId(workerContext.getConnectionInfo().getId()); + requestProto.setConnectionInfo(workerContext.getConnectionInfo().getProto()); //TODO set node status to requestProto.setStatus() return requestProto.build(); diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeStatusUpdater.java b/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeStatusUpdater.java index 2d7d0be6d8..dfcfd4f435 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeStatusUpdater.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/MockNodeStatusUpdater.java @@ -39,9 +39,9 @@ public class MockNodeStatusUpdater extends NodeStatusUpdater { private Map resources = Maps.newHashMap(); private MockResourceTracker resourceTracker; - public MockNodeStatusUpdater(CountDownLatch barrier, WorkerConnectionInfo connectionInfo, + public MockNodeStatusUpdater(CountDownLatch barrier, TajoWorker.WorkerContext workerContext, NodeResourceManager resourceManager) { - super(connectionInfo, resourceManager); + super(workerContext, resourceManager); this.barrier = barrier; this.resourceTracker = new MockResourceTracker(); } diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java index 68111b8aa6..2cee7d025f 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeResourceManager.java @@ -50,11 +50,11 @@ public class TestNodeResourceManager { private TaskExecutor taskExecutor; private AsyncDispatcher dispatcher; private AsyncDispatcher taskDispatcher; + private TajoWorker.WorkerContext workerContext; private CompositeService service; private int taskMemory; private TajoConf conf; - private WorkerConnectionInfo worker; @Before public void setup() { @@ -71,19 +71,26 @@ public void setup() { dispatcher = new AsyncDispatcher(); taskDispatcher = new AsyncDispatcher(); - final TajoWorker.WorkerContext workerContext = new MockWorkerContext() { - + workerContext = new MockWorkerContext() { + WorkerConnectionInfo workerConnectionInfo; @Override public TajoConf getConf() { return conf; } + + @Override + public WorkerConnectionInfo getConnectionInfo() { + if (workerConnectionInfo == null) { + workerConnectionInfo = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); + } + return workerConnectionInfo; + } }; taskManager = new MockTaskManager(new Semaphore(0), taskDispatcher, workerContext, dispatcher.getEventHandler()); taskExecutor = new MockTaskExecutor(new Semaphore(0), taskManager, dispatcher.getEventHandler()); resourceManager = new MockNodeResourceManager(new Semaphore(0), dispatcher, taskDispatcher.getEventHandler()); - worker = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); - statusUpdater = new MockNodeStatusUpdater(new CountDownLatch(0), worker, resourceManager); + statusUpdater = new MockNodeStatusUpdater(new CountDownLatch(0), workerContext, resourceManager); service = new CompositeService("MockService") { @Override @@ -195,8 +202,9 @@ public void testParallelRequest() throws Exception { TajoWorkerProtocol.RunExecutionBlockRequestProto.Builder ebRequestProto = TajoWorkerProtocol.RunExecutionBlockRequestProto.newBuilder(); ebRequestProto.setExecutionBlockId(ebId.getProto()) - .setQueryMaster(worker.getProto()) - .setNodeId(worker.getHost()+":" + worker.getQueryMasterPort()) + .setQueryMaster(workerContext.getConnectionInfo().getProto()) + .setNodeId(workerContext.getConnectionInfo().getHost() + ":" + + workerContext.getConnectionInfo().getQueryMasterPort()) .setContainerId("test") .setQueryContext(new QueryContext(conf).getProto()) .setPlanJson("test") diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java index 071a95d530..af40554f31 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestNodeStatusUpdater.java @@ -38,14 +38,29 @@ public class TestNodeStatusUpdater { private MockNodeStatusUpdater statusUpdater; private AsyncDispatcher dispatcher; private TajoConf conf; - private WorkerConnectionInfo worker; + private TajoWorker.WorkerContext workerContext; @Before public void setup() { conf = new TajoConf(); conf.set(CommonTestingUtil.TAJO_TEST_KEY, CommonTestingUtil.TAJO_TEST_TRUE); - worker = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); + workerContext = new MockWorkerContext() { + WorkerConnectionInfo workerConnectionInfo; + + @Override + public TajoConf getConf() { + return conf; + } + + @Override + public WorkerConnectionInfo getConnectionInfo() { + if (workerConnectionInfo == null) { + workerConnectionInfo = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); + } + return workerConnectionInfo; + } + }; conf.setIntVar(TajoConf.ConfVars.WORKER_HEARTBEAT_INTERVAL, 1000); dispatcher = new AsyncDispatcher(); @@ -67,25 +82,25 @@ public void tearDown() { @Test(timeout = 20000) public void testNodeMembership() throws Exception { CountDownLatch barrier = new CountDownLatch(1); - statusUpdater = new MockNodeStatusUpdater(barrier, worker, resourceManager); + statusUpdater = new MockNodeStatusUpdater(barrier, workerContext, resourceManager); statusUpdater.init(conf); statusUpdater.start(); MockNodeStatusUpdater.MockResourceTracker resourceTracker = statusUpdater.getResourceTracker(); barrier.await(); - assertTrue(resourceTracker.getTotalResource().containsKey(worker.getId())); + assertTrue(resourceTracker.getTotalResource().containsKey(workerContext.getConnectionInfo().getId())); assertEquals(resourceManager.getTotalResource(), - resourceTracker.getTotalResource().get(worker.getId())); + resourceTracker.getTotalResource().get(workerContext.getConnectionInfo().getId())); assertEquals(resourceManager.getAvailableResource(), - resourceTracker.getAvailableResource().get(worker.getId())); + resourceTracker.getAvailableResource().get(workerContext.getConnectionInfo().getId())); } @Test(timeout = 20000) public void testPing() throws Exception { CountDownLatch barrier = new CountDownLatch(2); - statusUpdater = new MockNodeStatusUpdater(barrier, worker, resourceManager); + statusUpdater = new MockNodeStatusUpdater(barrier, workerContext, resourceManager); statusUpdater.init(conf); statusUpdater.start(); @@ -102,7 +117,7 @@ public void testPing() throws Exception { @Test(timeout = 20000) public void testResourceReport() throws Exception { CountDownLatch barrier = new CountDownLatch(2); - statusUpdater = new MockNodeStatusUpdater(barrier, worker, resourceManager); + statusUpdater = new MockNodeStatusUpdater(barrier, workerContext, resourceManager); statusUpdater.init(conf); statusUpdater.start(); @@ -117,7 +132,7 @@ public void testResourceReport() throws Exception { @Test(timeout = 20000) public void testFlushResourceReport() throws Exception { CountDownLatch barrier = new CountDownLatch(2); - statusUpdater = new MockNodeStatusUpdater(barrier, worker, resourceManager); + statusUpdater = new MockNodeStatusUpdater(barrier, workerContext, resourceManager); statusUpdater.init(conf); statusUpdater.start(); diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java index 7e240e5ddf..98b187b013 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskExecutor.java @@ -70,7 +70,7 @@ public class TestTaskExecutor { private TaskExecutor taskExecutor; private AsyncDispatcher dispatcher; private AsyncDispatcher taskDispatcher; - private WorkerConnectionInfo worker; + private TajoWorker.WorkerContext workerContext; private CompositeService service; private TajoConf conf; @@ -84,12 +84,21 @@ public void setup() { dispatcher = new AsyncDispatcher(); taskDispatcher = new AsyncDispatcher(); - final TajoWorker.WorkerContext workerContext = new MockWorkerContext() { + workerContext = new MockWorkerContext() { + WorkerConnectionInfo workerConnectionInfo; @Override public TajoConf getConf() { return conf; } + + @Override + public WorkerConnectionInfo getConnectionInfo() { + if (workerConnectionInfo == null) { + workerConnectionInfo = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); + } + return workerConnectionInfo; + } }; barrier = new Semaphore(0); @@ -97,8 +106,7 @@ public TajoConf getConf() { taskManager = new MockTaskManager(new Semaphore(0), taskDispatcher, workerContext, dispatcher.getEventHandler()); taskExecutor = new TaskExecutor(barrier, taskManager, dispatcher.getEventHandler()); resourceManager = new MockNodeResourceManager(resourceManagerBarrier, dispatcher, taskDispatcher.getEventHandler()); - worker = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); - statusUpdater = new MockNodeStatusUpdater(new CountDownLatch(0), worker, resourceManager); + statusUpdater = new MockNodeStatusUpdater(new CountDownLatch(0), workerContext, resourceManager); service = new CompositeService("MockService") { @Override @@ -139,8 +147,9 @@ public void testTaskRequest() throws Exception { ExecutionBlockId ebId = QueryIdFactory.newExecutionBlockId(qid, 1); ebRequestProto.setExecutionBlockId(ebId.getProto()) - .setQueryMaster(worker.getProto()) - .setNodeId(worker.getHost()+":" + worker.getQueryMasterPort()) + .setQueryMaster(workerContext.getConnectionInfo().getProto()) + .setNodeId(workerContext.getConnectionInfo().getHost() + ":" + + workerContext.getConnectionInfo().getQueryMasterPort()) .setContainerId("test") .setQueryContext(new QueryContext(conf).getProto()) .setPlanJson("test") @@ -178,8 +187,9 @@ public void testTaskException() throws Exception { ExecutionBlockId ebId = QueryIdFactory.newExecutionBlockId(qid, 1); ebRequestProto.setExecutionBlockId(ebId.getProto()) - .setQueryMaster(worker.getProto()) - .setNodeId(worker.getHost()+":" + worker.getQueryMasterPort()) + .setQueryMaster(workerContext.getConnectionInfo().getProto()) + .setNodeId(workerContext.getConnectionInfo().getHost()+":" + + workerContext.getConnectionInfo().getQueryMasterPort()) .setContainerId("test") .setQueryContext(new QueryContext(conf).getProto()) .setPlanJson("test") diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java index 6cdcf14e31..8bca489e20 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestTaskManager.java @@ -49,7 +49,7 @@ public class TestTaskManager { private TaskExecutor taskExecutor; private AsyncDispatcher dispatcher; private AsyncDispatcher taskDispatcher; - private WorkerConnectionInfo worker; + private TajoWorker.WorkerContext workerContext; private CompositeService service; private int taskMemory; @@ -71,19 +71,27 @@ public void setup() { dispatcher = new AsyncDispatcher(); taskDispatcher = new AsyncDispatcher(); - final TajoWorker.WorkerContext workerContext = new MockWorkerContext() { + workerContext = new MockWorkerContext() { + WorkerConnectionInfo workerConnectionInfo; @Override public TajoConf getConf() { return conf; } + + @Override + public WorkerConnectionInfo getConnectionInfo() { + if (workerConnectionInfo == null) { + workerConnectionInfo = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); + } + return workerConnectionInfo; + } }; barrier = new Semaphore(0); taskManager = new MockTaskManager(barrier, taskDispatcher, workerContext, dispatcher.getEventHandler()); taskExecutor = new MockTaskExecutor(new Semaphore(0), taskManager, dispatcher.getEventHandler()); resourceManager = new NodeResourceManager(dispatcher, taskDispatcher.getEventHandler()); - worker = new WorkerConnectionInfo("host", 28091, 28092, 21000, 28093, 28080); - statusUpdater = new MockNodeStatusUpdater(new CountDownLatch(0), worker, resourceManager); + statusUpdater = new MockNodeStatusUpdater(new CountDownLatch(0), workerContext, resourceManager); service = new CompositeService("MockService") { @Override @@ -124,8 +132,9 @@ public void testExecutionBlockStart() throws Exception { ExecutionBlockId ebId = QueryIdFactory.newExecutionBlockId(qid, 1); ebRequestProto.setExecutionBlockId(ebId.getProto()) - .setQueryMaster(worker.getProto()) - .setNodeId(worker.getHost()+":" + worker.getQueryMasterPort()) + .setQueryMaster(workerContext.getConnectionInfo().getProto()) + .setNodeId(workerContext.getConnectionInfo().getHost() + ":" + + workerContext.getConnectionInfo().getQueryMasterPort()) .setContainerId("test") .setQueryContext(new QueryContext(conf).getProto()) .setPlanJson("test") @@ -155,8 +164,9 @@ public void testExecutionBlockStop() throws Exception { ExecutionBlockId ebId = QueryIdFactory.newExecutionBlockId(qid, 1); ebRequestProto.setExecutionBlockId(ebId.getProto()) - .setQueryMaster(worker.getProto()) - .setNodeId(worker.getHost()+":" + worker.getQueryMasterPort()) + .setQueryMaster(workerContext.getConnectionInfo().getProto()) + .setNodeId(workerContext.getConnectionInfo().getHost()+":" + + workerContext.getConnectionInfo().getQueryMasterPort()) .setContainerId("test") .setQueryContext(new QueryContext(conf).getProto()) .setPlanJson("test") From be992854a846ecc9f10486cb414b7132942b7237 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Mon, 1 Jun 2015 12:14:43 +0900 Subject: [PATCH 19/21] TAJO-1615 --- .../src/main/java/org/apache/tajo/worker/TaskExecutor.java | 3 +++ .../src/main/java/org/apache/tajo/worker/TaskManager.java | 3 +++ 2 files changed, 6 insertions(+) diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java index 87f18bbc7a..0683f83d10 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java @@ -43,6 +43,9 @@ import java.util.concurrent.*; import java.util.concurrent.atomic.AtomicInteger; +/** + * TaskExecutor uses a number of threads equal to the number of slots available for running tasks on the Worker + */ public class TaskExecutor extends AbstractService implements EventHandler { private static final Log LOG = LogFactory.getLog(TaskExecutor.class); diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java index bbf188e14b..7990a7213e 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskManager.java @@ -36,6 +36,9 @@ import java.io.IOException; import java.util.*; +/** + * A TaskManager is responsible for managing executionBlock resource and tasks. + * */ public class TaskManager extends AbstractService implements EventHandler { private static final Log LOG = LogFactory.getLog(TaskManager.class); From 7b1785ef9280490fe70bef92340e053df78be504 Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Mon, 1 Jun 2015 12:41:17 +0900 Subject: [PATCH 20/21] TAJO-1615 --- tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java | 3 +-- .../src/main/java/org/apache/tajo/worker/TaskExecutor.java | 3 ++- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java index c5ced0713b..3f350c3143 100644 --- a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java +++ b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java @@ -216,8 +216,7 @@ public static enum ConfVars implements ConfigKey { PULLSERVER_PORT("tajo.pullserver.port", 0, Validators.range("0", "65535")), SHUFFLE_SSL_ENABLED_KEY("tajo.pullserver.ssl.enabled", false, Validators.bool()), SHUFFLE_FILE_FORMAT("tajo.shuffle.file-format", "RAW"), - SHUFFLE_FETCHER_PARALLEL_EXECUTION_MAX_NUM("tajo.shuffle.fetcher.parallel-execution.max-num", - Runtime.getRuntime().availableProcessors() * 2, Validators.min("1")), + SHUFFLE_FETCHER_PARALLEL_EXECUTION_MAX_NUM("tajo.shuffle.fetcher.parallel-execution.max-num", 2), SHUFFLE_FETCHER_CHUNK_MAX_SIZE("tajo.shuffle.fetcher.chunk.max-size", 8192), SHUFFLE_FETCHER_READ_TIMEOUT("tajo.shuffle.fetcher.read.timeout-sec", 120), SHUFFLE_FETCHER_READ_RETRY_MAX_NUM("tajo.shuffle.fetcher.read.retry.max-num", 20), diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java index 0683f83d10..299952e7ae 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskExecutor.java @@ -85,7 +85,8 @@ protected void serviceStart() throws Exception { this.threadPool = Executors.newFixedThreadPool(nThreads, new ThreadFactoryBuilder().setNameFormat("Task executor #%d").build()); - int maxFetcherThreads = tajoConf.getIntVar(ConfVars.SHUFFLE_FETCHER_PARALLEL_EXECUTION_MAX_NUM); + //TODO move to tajoConf.getIntVar(ConfVars.SHUFFLE_FETCHER_PARALLEL_EXECUTION_MAX_NUM); + int maxFetcherThreads = Runtime.getRuntime().availableProcessors() * 2; this.fetcherExecutor = new ThreadPoolExecutor(Math.min(nThreads, maxFetcherThreads), maxFetcherThreads, 60L, TimeUnit.SECONDS, From cd5b99e003f34b52257235deb1e1b8795b141e6b Mon Sep 17 00:00:00 2001 From: Jinho Kim Date: Mon, 1 Jun 2015 14:06:06 +0900 Subject: [PATCH 21/21] add comments in events --- .../java/org/apache/tajo/worker/event/NodeResourceEvent.java | 3 +++ .../java/org/apache/tajo/worker/event/NodeStatusEvent.java | 3 +++ .../java/org/apache/tajo/worker/event/TaskExecutorEvent.java | 2 ++ .../java/org/apache/tajo/worker/event/TaskManagerEvent.java | 1 + 4 files changed, 9 insertions(+) diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceEvent.java index 69c1d0c1e4..6fd2e0d517 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeResourceEvent.java @@ -21,8 +21,11 @@ import org.apache.hadoop.yarn.event.AbstractEvent; public class NodeResourceEvent extends AbstractEvent { + //consumer: NodeResourceManager public enum EventType { + // producer: TajoWorkerManagerService ALLOCATE, + // producer: TaskExecutor DEALLOCATE } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeStatusEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeStatusEvent.java index ec604fe76e..9eb8ae9d7b 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeStatusEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/NodeStatusEvent.java @@ -23,8 +23,11 @@ public class NodeStatusEvent extends AbstractEvent { + // consumer: NodeStatusUpdater public enum EventType { + // producer: NodeResourceManager REPORT_RESOURCE, + // producer: TaskManager FLUSH_REPORTS } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskExecutorEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskExecutorEvent.java index 1ec9b8bec1..c609c67150 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskExecutorEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskExecutorEvent.java @@ -22,6 +22,8 @@ import org.apache.tajo.TaskAttemptId; public class TaskExecutorEvent extends AbstractEvent { + + // producer: NodeResourceManager, consumer: TaskExecutorEvent public enum EventType { START, KILL, diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskManagerEvent.java b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskManagerEvent.java index dcf33769d7..39b541b097 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskManagerEvent.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/event/TaskManagerEvent.java @@ -23,6 +23,7 @@ import org.apache.tajo.TaskAttemptId; public class TaskManagerEvent extends AbstractEvent { + // producer: NodeResourceManager, consumer: TaskManager public enum EventType { EB_START, EB_STOP