From 1392ea9fef37f80e272a083563b6b798f519d197 Mon Sep 17 00:00:00 2001 From: Laszlo Bodor Date: Fri, 5 Sep 2025 17:47:48 +0200 Subject: [PATCH 1/5] TEZ-4007: Introduce AmExtensions and Zookeeper-based FrameworkServices - initial patch --- pom.xml | 6 + tez-api/pom.xml | 7 +- .../apache/tez/client/FrameworkClient.java | 8 + .../java/org/apache/tez/client/TezClient.java | 7 +- .../org/apache/tez/client/TezClientUtils.java | 44 ++-- .../apache/tez/client/registry/AMRecord.java | 26 +- .../tez/client/registry/AMRegistry.java | 94 +++++++ .../tez/client/registry/AMRegistryClient.java | 55 +++++ .../registry/AMRegistryClientListener.java | 24 ++ .../zookeeper/ZkAMRegistryClient.java | 188 ++++++++++++++ .../client/registry/zookeeper/ZkConfig.java | 91 +++++++ .../registry/zookeeper/ZkFrameworkClient.java | 129 ++++++++++ .../apache/tez/dag/api/TezConfiguration.java | 58 ++++- .../org/apache/tez/dag/api/TezConstants.java | 20 +- .../dag/api/client/rpc/DAGClientRPCImpl.java | 1 - .../ClientFrameworkService.java | 35 +++ .../tez/frameworkplugins/FrameworkMode.java | 40 +++ .../frameworkplugins/FrameworkService.java | 24 ++ .../tez/frameworkplugins/FrameworkUtils.java | 98 ++++++++ ...eeperStandaloneClientFrameworkService.java | 26 ++ .../org/apache/tez/client/TestTezClient.java | 107 +++++++- .../apache/tez/client/TestTezClientUtils.java | 18 ++ .../apache/tez/common/TezUtilsInternal.java | 18 ++ tez-dag/pom.xml | 14 ++ .../org/apache/tez/client/LocalClient.java | 2 +- .../tez/dag/api/client/DAGClientServer.java | 6 +- .../registry/zookeeper/ZkAMRegistry.java | 168 +++++++++++++ .../org/apache/tez/dag/app/DAGAppMaster.java | 229 ++++++++++-------- .../apache/tez/dag/app/LocalDAGAppMaster.java | 2 +- .../apache/tez/dag/app/RecoveryParser.java | 1 + .../tez/dag/app/TezTaskCommunicatorImpl.java | 3 +- .../apache/tez/dag/app/dag/impl/DAGImpl.java | 50 ++-- .../tez/dag/app/dag/impl/TaskAttemptImpl.java | 19 +- .../apache/tez/dag/app/dag/impl/TaskImpl.java | 5 +- .../app/launcher/TezLocalCacheManager.java | 6 +- .../tez/dag/app/rm/TaskSchedulerManager.java | 2 +- .../apache/tez/dag/utils/AMRegistryUtils.java | 45 ---- .../tez/frameworkplugins/AmExtensions.java | 63 +++++ .../ServerFrameworkService.java | 32 +++ .../zookeeper/ZkStandaloneAmExtensions.java | 104 ++++++++ ...eeperStandaloneServerFrameworkService.java | 39 +++ .../api/client/registry/TestAMRegistry.java | 100 -------- .../apache/tez/dag/app/MockDAGAppMaster.java | 2 +- .../apache/tez/dag/app/TestDAGAppMaster.java | 6 +- .../dag/app/dag/impl/TestVertexManager.java | 1 - .../dag/app/rm/TestTaskSchedulerHelpers.java | 1 - .../org/apache/tez/examples/ExampleBase.java | 40 +++ .../tez/examples/StandaloneWordCount.java | 128 ++++++++++ .../examples/StandaloneWordCountDriver.java | 109 +++++++++ .../apache/tez/mapreduce/SumProcessor.java | 40 +++ .../apache/tez/mapreduce/TokenProcessor.java | 45 ++++ .../apache/tez/runtime/library/FakeInput.java | 80 ++++++ .../runtime/library/FakeInputInitializer.java | 43 ++++ .../tez/runtime/library/FakeOutput.java | 58 +++++ .../runtime/library/FakeOutputCommitter.java | 40 +++ 55 files changed, 2258 insertions(+), 349 deletions(-) create mode 100644 tez-api/src/main/java/org/apache/tez/client/registry/AMRegistry.java create mode 100644 tez-api/src/main/java/org/apache/tez/client/registry/AMRegistryClient.java create mode 100644 tez-api/src/main/java/org/apache/tez/client/registry/AMRegistryClientListener.java create mode 100644 tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/ZkAMRegistryClient.java create mode 100644 tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/ZkConfig.java create mode 100644 tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/ZkFrameworkClient.java create mode 100644 tez-api/src/main/java/org/apache/tez/frameworkplugins/ClientFrameworkService.java create mode 100644 tez-api/src/main/java/org/apache/tez/frameworkplugins/FrameworkMode.java create mode 100644 tez-api/src/main/java/org/apache/tez/frameworkplugins/FrameworkService.java create mode 100644 tez-api/src/main/java/org/apache/tez/frameworkplugins/FrameworkUtils.java create mode 100644 tez-api/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZookeeperStandaloneClientFrameworkService.java create mode 100644 tez-dag/src/main/java/org/apache/tez/dag/api/client/registry/zookeeper/ZkAMRegistry.java delete mode 100644 tez-dag/src/main/java/org/apache/tez/dag/utils/AMRegistryUtils.java create mode 100644 tez-dag/src/main/java/org/apache/tez/frameworkplugins/AmExtensions.java create mode 100644 tez-dag/src/main/java/org/apache/tez/frameworkplugins/ServerFrameworkService.java create mode 100644 tez-dag/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZkStandaloneAmExtensions.java create mode 100644 tez-dag/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZookeeperStandaloneServerFrameworkService.java delete mode 100644 tez-dag/src/test/java/org/apache/tez/dag/api/client/registry/TestAMRegistry.java create mode 100644 tez-examples/src/main/java/org/apache/tez/examples/ExampleBase.java create mode 100644 tez-examples/src/main/java/org/apache/tez/examples/StandaloneWordCount.java create mode 100644 tez-examples/src/main/java/org/apache/tez/examples/StandaloneWordCountDriver.java create mode 100644 tez-mapreduce/src/main/java/org/apache/tez/mapreduce/SumProcessor.java create mode 100644 tez-mapreduce/src/main/java/org/apache/tez/mapreduce/TokenProcessor.java create mode 100644 tez-runtime-library/src/main/java/org/apache/tez/runtime/library/FakeInput.java create mode 100644 tez-runtime-library/src/main/java/org/apache/tez/runtime/library/FakeInputInitializer.java create mode 100644 tez-runtime-library/src/main/java/org/apache/tez/runtime/library/FakeOutput.java create mode 100644 tez-runtime-library/src/main/java/org/apache/tez/runtime/library/FakeOutputCommitter.java diff --git a/pom.xml b/pom.xml index 0e6955030c..fd68f22f20 100644 --- a/pom.xml +++ b/pom.xml @@ -69,6 +69,7 @@ 2.16.0 2.6 ${user.home}/clover.license + 2.7.1 3.2.0 3.8.1 4.9.3 @@ -736,6 +737,11 @@ + + org.apache.hadoop + hadoop-registry + ${hadoop.version} + org.apache.hadoop hadoop-hdfs diff --git a/tez-api/pom.xml b/tez-api/pom.xml index ff1f9d2e6a..b6495f693f 100644 --- a/tez-api/pom.xml +++ b/tez-api/pom.xml @@ -76,7 +76,6 @@ org.apache.hadoop hadoop-registry - ${hadoop.version} org.apache.commons @@ -124,6 +123,12 @@ org.xerial.snappy snappy-java + + org.apache.curator + curator-test + ${curator.version} + test + diff --git a/tez-api/src/main/java/org/apache/tez/client/FrameworkClient.java b/tez-api/src/main/java/org/apache/tez/client/FrameworkClient.java index 9b7a32b61f..f5037e7f2d 100644 --- a/tez-api/src/main/java/org/apache/tez/client/FrameworkClient.java +++ b/tez-api/src/main/java/org/apache/tez/client/FrameworkClient.java @@ -19,6 +19,7 @@ package org.apache.tez.client; import java.io.IOException; +import java.util.Optional; import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.conf.Configuration; @@ -46,6 +47,8 @@ import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.ShutdownSessionRequestProto; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.SubmitDAGRequestProto; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.SubmitDAGResponseProto; +import org.apache.tez.frameworkplugins.ClientFrameworkService; +import org.apache.tez.frameworkplugins.FrameworkUtils; import com.google.protobuf.ServiceException; @@ -57,6 +60,9 @@ public abstract class FrameworkClient { protected static final Logger LOG = LoggerFactory.getLogger(FrameworkClient.class); public static FrameworkClient createFrameworkClient(TezConfiguration tezConf) { + Optional pluginClient = + FrameworkUtils.get(ClientFrameworkService.class, tezConf) + .flatMap(framework -> framework.createOrGetFrameworkClient(tezConf)); boolean isLocal = tezConf.getBoolean(TezConfiguration.TEZ_LOCAL_MODE, TezConfiguration.TEZ_LOCAL_MODE_DEFAULT); if (isLocal) { @@ -65,6 +71,8 @@ public static FrameworkClient createFrameworkClient(TezConfiguration tezConf) { } catch (TezReflectionException e) { throw new TezUncheckedException("Fail to create LocalClient", e); } + } else if (pluginClient.isPresent()) { + return pluginClient.get(); } return new TezYarnClient(YarnClient.createYarnClient()); } diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClient.java b/tez-api/src/main/java/org/apache/tez/client/TezClient.java index e6c852a1a3..0e4167e323 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClient.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClient.java @@ -484,7 +484,9 @@ public synchronized TezClient getClient(ApplicationId appId) throws TezException } private void startFrameworkClient() { - frameworkClient = createFrameworkClient(); + if (frameworkClient == null) { + frameworkClient = createFrameworkClient(); + } frameworkClient.init(amConfig.getTezConfiguration()); frameworkClient.start(); } @@ -670,6 +672,8 @@ private DAGClient submitDAGSession(DAG dag) throws TezException, IOException { } Map tezJarResources = getTezJarResources(sessionCredentials); + + DAGPlan dagPlan = TezClientUtils.prepareAndCreateDAGPlan(dag, amConfig, tezJarResources, usingTezArchiveDeploy, sessionCredentials, servicePluginsDescriptor, javaOptsChecker); @@ -684,6 +688,7 @@ private DAGClient submitDAGSession(DAG dag) throws TezException, IOException { // if request size exceeds maxSubmitDAGRequestSizeThroughIPC, we serialize them to HDFS SubmitDAGRequestProto request = requestBuilder.build(); + if (request.getSerializedSize() > maxSubmitDAGRequestSizeThroughIPC) { Path dagPlanPath = new Path(TezCommonUtils.getTezSystemStagingPath(amConfig.getTezConfiguration(), sessionAppId.toString()), TezConstants.TEZ_PB_PLAN_BINARY_NAME + diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java index 0a3c14fd69..92e44f17bc 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java @@ -149,6 +149,8 @@ private static Path getPath(String configUri) { } } + + /** * Setup LocalResource map for Tez jars based on provided Configuration * @@ -186,7 +188,7 @@ static boolean setupTezJarsLocalResources(TezConfiguration conf, + conf.get(TezConfiguration.TEZ_LIB_URIS_CLASSPATH)); usingTezArchive = addLocalResources(conf, tezJarUris, - tezJarResources, credentials); + tezJarResources, credentials); if (tezJarResources.isEmpty()) { throw new TezUncheckedException( @@ -263,8 +265,8 @@ private static boolean addLocalResources(Configuration conf, // Add URI fragment or just the filename Path name = new Path((null == u.getFragment()) - ? p.getName() - : u.getFragment()); + ? p.getName() + : u.getFragment()); if (name.isAbsolute()) { throw new IllegalArgumentException("Resource name must be " + "relative, not absolute: " + name @@ -577,7 +579,7 @@ public static ApplicationSubmissionContext createApplicationSubmissionContext( // don't overwrite existing conf, needed for TezClient.getClient() so existing containers have stable resource fingerprints if(!binaryConfPath.getFileSystem(tezConf).exists(binaryConfPath)) { ConfigurationProto finalConfProto = createFinalConfProtoForApp(tezConf, - servicePluginsDescriptor); + servicePluginsDescriptor); FSDataOutputStream amConfPBOutBinaryStream = null; try { @@ -618,12 +620,12 @@ public static ApplicationSubmissionContext createApplicationSubmissionContext( } LocalResource sessionJarsPBLRsrc = - TezClientUtils.createLocalResource(fs, - sessionJarsPath, LocalResourceType.FILE, - LocalResourceVisibility.APPLICATION); + TezClientUtils.createLocalResource(fs, + sessionJarsPath, LocalResourceType.FILE, + LocalResourceVisibility.APPLICATION); amLocalResources.put( - TezConstants.TEZ_AM_LOCAL_RESOURCES_PB_FILE_NAME, - sessionJarsPBLRsrc); + TezConstants.TEZ_AM_LOCAL_RESOURCES_PB_FILE_NAME, + sessionJarsPBLRsrc); String user = UserGroupInformation.getCurrentUser().getShortUserName(); ACLManager aclManager = new ACLManager(user, amConfig.getTezConfiguration()); @@ -655,9 +657,9 @@ public static ApplicationSubmissionContext createApplicationSubmissionContext( } amLocalResources.put(TezConstants.TEZ_PB_PLAN_BINARY_NAME, - TezClientUtils.createLocalResource(fs, - binaryPath, LocalResourceType.FILE, - LocalResourceVisibility.APPLICATION)); + TezClientUtils.createLocalResource(fs, + binaryPath, LocalResourceType.FILE, + LocalResourceVisibility.APPLICATION)); if (Level.DEBUG.isGreaterOrEqual(Level.toLevel(amLogLevel))) { Path textPath = localizeDagPlanAsText(dagPB, fs, amConfig, strAppId, tezSysStagingPath); @@ -688,7 +690,7 @@ public static ApplicationSubmissionContext createApplicationSubmissionContext( Collection tagsFromConf = amConfig.getTezConfiguration().getTrimmedStringCollection( - TezConfiguration.TEZ_APPLICATION_TAGS); + TezConfiguration.TEZ_APPLICATION_TAGS); appContext.setApplicationType(TezConstants.TEZ_APPLICATION_TYPE); if (tagsFromConf != null && !tagsFromConf.isEmpty()) { @@ -709,8 +711,8 @@ public static ApplicationSubmissionContext createApplicationSubmissionContext( appContext.setAMContainerSpec(amContainer); appContext.setMaxAppAttempts( - amConfig.getTezConfiguration().getInt(TezConfiguration.TEZ_AM_MAX_APP_ATTEMPTS, - TezConfiguration.TEZ_AM_MAX_APP_ATTEMPTS_DEFAULT)); + amConfig.getTezConfiguration().getInt(TezConfiguration.TEZ_AM_MAX_APP_ATTEMPTS, + TezConfiguration.TEZ_AM_MAX_APP_ATTEMPTS_DEFAULT)); return appContext; @@ -848,7 +850,7 @@ public static void addLog4jSystemProperties(String logLevel, } public static ConfigurationProto createFinalConfProtoForApp(Configuration amConf, - ServicePluginsDescriptor servicePluginsDescriptor) { + ServicePluginsDescriptor servicePluginsDescriptor) { assert amConf != null; ConfigurationProto.Builder builder = ConfigurationProto.newBuilder(); for (Entry entry : amConf) { @@ -949,7 +951,7 @@ static DAGClientAMProtocolBlockingPB getAMProxy(FrameworkClient frameworkClient, + ", trackingUrl=" + appReport.getTrackingUrl() + ", diagnostics=" + (appReport.getDiagnostics() != null ? appReport.getDiagnostics() - : TezClient.NO_CLUSTER_DIAGNOSTICS_MSG); + : TezClient.NO_CLUSTER_DIAGNOSTICS_MSG); LOG.info(msg); throw new SessionNotRunning(msg); } @@ -1015,7 +1017,7 @@ static void createSessionToken(String tokenIdentifier, public static String maybeAddDefaultMemoryJavaOpts(String javaOpts, Resource resource, double maxHeapFactor) { if ((javaOpts != null && !javaOpts.isEmpty() - && (javaOpts.contains("-Xmx") || javaOpts.contains("-Xms"))) + && (javaOpts.contains("-Xmx") || javaOpts.contains("-Xms"))) || (resource.getMemory() <= 0)) { return javaOpts; } @@ -1026,8 +1028,8 @@ public static String maybeAddDefaultMemoryJavaOpts(String javaOpts, Resource res if (Double.parseDouble("-1") == maxHeapFactor) { maxHeapFactor = resource.getMemory() < TezConstants.TEZ_CONTAINER_SMALL_SLAB_BOUND_MB - ? TezConstants.TEZ_CONTAINER_MAX_JAVA_HEAP_FRACTION_SMALL_SLAB - : TezConstants.TEZ_CONTAINER_MAX_JAVA_HEAP_FRACTION_LARGE_SLAB; + ? TezConstants.TEZ_CONTAINER_MAX_JAVA_HEAP_FRACTION_SMALL_SLAB + : TezConstants.TEZ_CONTAINER_MAX_JAVA_HEAP_FRACTION_LARGE_SLAB; } int maxMemory = (int)(resource.getMemory() * maxHeapFactor); maxMemory = maxMemory <= 0 ? 1 : maxMemory; @@ -1037,7 +1039,7 @@ public static String maybeAddDefaultMemoryJavaOpts(String javaOpts, Resource res } private static boolean checkAncestorPermissionsForAllUsers(Configuration conf, Path pathComponent, - FsAction permission) throws IOException { + FsAction permission) throws IOException { FileSystem fs = pathComponent.getFileSystem(conf); if (Shell.WINDOWS && fs instanceof LocalFileSystem) { diff --git a/tez-api/src/main/java/org/apache/tez/client/registry/AMRecord.java b/tez-api/src/main/java/org/apache/tez/client/registry/AMRecord.java index 8453f9836c..2f5ab978d5 100644 --- a/tez-api/src/main/java/org/apache/tez/client/registry/AMRecord.java +++ b/tez-api/src/main/java/org/apache/tez/client/registry/AMRecord.java @@ -39,12 +39,12 @@ public class AMRecord { private static final String APP_ID_RECORD_KEY = "appId"; private static final String HOST_RECORD_KEY = "host"; private static final String PORT_RECORD_KEY = "port"; - private static final String OPAQUE_ID_KEY = "id"; + private static final String EXTERNAL_ID_KEY = "externalId"; private final ApplicationId appId; private final String host; private final int port; - private final String id; + private final String externalId; /** * Creates a new {@code AMRecord} with the given application ID, host, port, and identifier. @@ -57,14 +57,14 @@ public class AMRecord { * @param appId the {@link ApplicationId} of the Tez application * @param host the hostname where the Application Master is running * @param port the port number on which the Application Master is listening - * @param id an opaque identifier for the record; if {@code null}, defaults to an empty string + * @param externalId an opaque identifier for the record; if {@code null}, defaults to an empty string */ - public AMRecord(ApplicationId appId, String host, int port, String id) { + public AMRecord(ApplicationId appId, String host, int port, String externalId) { this.appId = appId; this.host = host; this.port = port; - //If id is not provided, convert to empty string - this.id = (id == null) ? "" : id; + //externalId is optional, if not provided, convert to empty string + this.externalId = (externalId == null) ? "" : externalId; } /** @@ -81,7 +81,7 @@ public AMRecord(AMRecord other) { this.appId = other.getApplicationId(); this.host = other.getHost(); this.port = other.getPort(); - this.id = other.getId(); + this.externalId = other.getExternalId(); } /** @@ -99,7 +99,7 @@ public AMRecord(ServiceRecord serviceRecord) { this.appId = ApplicationId.fromString(serviceRecord.get(APP_ID_RECORD_KEY)); this.host = serviceRecord.get(HOST_RECORD_KEY); this.port = Integer.parseInt(serviceRecord.get(PORT_RECORD_KEY)); - this.id = serviceRecord.get(OPAQUE_ID_KEY); + this.externalId = serviceRecord.get(EXTERNAL_ID_KEY); } public ApplicationId getApplicationId() { @@ -114,8 +114,8 @@ public int getPort() { return port; } - public String getId() { - return id; + public String getExternalId() { + return externalId; } @Override @@ -127,7 +127,7 @@ public boolean equals(Object other) { return appId.equals(otherRecord.appId) && host.equals(otherRecord.host) && port == otherRecord.port - && id.equals(otherRecord.id); + && externalId.equals(otherRecord.externalId); } else { return false; } @@ -152,12 +152,12 @@ public ServiceRecord toServiceRecord() { serviceRecord.set(APP_ID_RECORD_KEY, appId); serviceRecord.set(HOST_RECORD_KEY, host); serviceRecord.set(PORT_RECORD_KEY, port); - serviceRecord.set(OPAQUE_ID_KEY, id); + serviceRecord.set(EXTERNAL_ID_KEY, externalId); return serviceRecord; } @Override public int hashCode() { - return Objects.hash(appId, host, port, id); + return Objects.hash(appId, host, port, externalId); } } diff --git a/tez-api/src/main/java/org/apache/tez/client/registry/AMRegistry.java b/tez-api/src/main/java/org/apache/tez/client/registry/AMRegistry.java new file mode 100644 index 0000000000..bb1b16da7c --- /dev/null +++ b/tez-api/src/main/java/org/apache/tez/client/registry/AMRegistry.java @@ -0,0 +1,94 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.client.registry; + +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.service.ServiceStateException; +import org.apache.hadoop.yarn.api.records.ApplicationId; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Base class for AMRegistry implementation + * Implementation class is configured by tez.am.registry.class + * Implementations should implement relevant service lifecycle operations: + * init, serviceStart, serviceStop, etc.. + * + * init/serviceStart will be invoked during DAGAppMaster.serviceInit + * + * serviceStop will invoked on DAGAppMaster shutdown + */ +public abstract class AMRegistry extends AbstractService { + + private static final Logger LOG = LoggerFactory.getLogger(AMRegistry.class); + protected List amRecords = new ArrayList<>(); + + @Override + public void init(Configuration conf) { + try { + this.serviceInit(conf); + } catch (Exception e) { + LOG.error("Failed to init AMRegistry: name={}, type={}", getName(), getClass().getName()); + throw ServiceStateException.convert(e); + } + } + + @Override + public void start() { + try { + this.serviceStart(); + } catch(Exception e) { + LOG.error("Failed to start AMRegistry: name={}, type={}", getName(), getClass().getName()); + throw ServiceStateException.convert(e); + } + } + + /* Implementations should provide a public no-arg constructor */ + protected AMRegistry(String name) { + super(name); + } + + /* Under typical usage, add will be called once automatically with an AMRecord + for the DAGClientServer servicing an AM + */ + public void add(AMRecord server) throws Exception { + amRecords.add(server); + } + + public abstract void remove(AMRecord server) throws Exception; + + public Optional generateNewId() throws Exception { + return Optional.empty(); + } + + public abstract AMRecord createAmRecord(ApplicationId appId, String hostName, int port); + + @Override public void serviceStop() throws Exception { + List records = new ArrayList<>(amRecords); + for(AMRecord record : records) { + remove(record); + } + } +} diff --git a/tez-api/src/main/java/org/apache/tez/client/registry/AMRegistryClient.java b/tez-api/src/main/java/org/apache/tez/client/registry/AMRegistryClient.java new file mode 100644 index 0000000000..9a34fbb78e --- /dev/null +++ b/tez-api/src/main/java/org/apache/tez/client/registry/AMRegistryClient.java @@ -0,0 +1,55 @@ +/** + * 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.tez.client.registry; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +/** + * Interface for client-side AM discovery + */ +public abstract class AMRegistryClient implements Closeable { + + protected List listeners = new ArrayList<>(); + + //Get AM info given an appId + public abstract AMRecord getRecord(String appId) throws IOException; + + //Get all AM infos in the registry + public abstract List getAllRecords() throws IOException; + + public synchronized void addListener(AMRegistryClientListener listener) { + listeners.add(listener); + } + + protected synchronized void notifyOnAdded(AMRecord record) { + for(AMRegistryClientListener listener : listeners) { + listener.onAdd(record); + } + } + + protected synchronized void notifyOnRemoved(AMRecord record) { + for(AMRegistryClientListener listener : listeners) { + listener.onRemove(record); + } + } + +} diff --git a/tez-api/src/main/java/org/apache/tez/client/registry/AMRegistryClientListener.java b/tez-api/src/main/java/org/apache/tez/client/registry/AMRegistryClientListener.java new file mode 100644 index 0000000000..527d821bde --- /dev/null +++ b/tez-api/src/main/java/org/apache/tez/client/registry/AMRegistryClientListener.java @@ -0,0 +1,24 @@ +/** + * 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.tez.client.registry; + +public interface AMRegistryClientListener { + void onAdd(AMRecord record); + void onRemove(AMRecord record); +} diff --git a/tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/ZkAMRegistryClient.java b/tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/ZkAMRegistryClient.java new file mode 100644 index 0000000000..41ce21eb31 --- /dev/null +++ b/tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/ZkAMRegistryClient.java @@ -0,0 +1,188 @@ +/** + * 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.tez.client.registry.zookeeper; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; + +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.imps.CuratorFrameworkState; +import org.apache.curator.framework.recipes.cache.ChildData; +import org.apache.curator.framework.recipes.cache.PathChildrenCache; +import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent; +import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.registry.client.binding.RegistryUtils; +import org.apache.hadoop.registry.client.types.ServiceRecord; +import org.apache.tez.client.registry.AMRecord; +import org.apache.tez.client.registry.AMRegistryClient; +import org.apache.tez.client.registry.AMRegistryClientListener; +import org.apache.tez.dag.api.TezConfiguration; + +import com.google.common.base.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Curator/Zookeeper impl of AMRegistryClient +*/ +@InterfaceAudience.Public +public class ZkAMRegistryClient extends AMRegistryClient { + private static final Logger LOG = LoggerFactory.getLogger(ZkAMRegistryClient.class); + + private final Configuration conf; + + //Cache of known AMs + private ConcurrentHashMap amRecordCache = new ConcurrentHashMap<>(); + private CuratorFramework client; + private PathChildrenCache cache; + + private static Map INSTANCES = new HashMap<>(); + + public static synchronized ZkAMRegistryClient getClient(final Configuration conf) { + String namespace = conf.get(TezConfiguration.TEZ_AM_REGISTRY_NAMESPACE); + ZkAMRegistryClient registry = INSTANCES.get(namespace); + if (registry == null) { + registry = new ZkAMRegistryClient(conf); + INSTANCES.put(namespace, registry); + } + LOG.info("Returning tez AM registry ({}) for namespace '{}'", System.identityHashCode(registry), namespace); + return registry; + } + + private ZkAMRegistryClient(final Configuration conf) { + this.conf = conf; + } + + public void start() throws Exception { + ZkConfig zkConf = new ZkConfig(this.conf); + client = zkConf.createCuratorFramework(); + cache = new PathChildrenCache(client, zkConf.getZkNamespace(), true); + client.start(); + cache.start(PathChildrenCache.StartMode.BUILD_INITIAL_CACHE); + for (ChildData childData : cache.getCurrentData()) { + AMRecord amRecord = getAMRecord(childData); + if (amRecord != null) { + amRecordCache.put(amRecord.getApplicationId().toString(), amRecord); + } + } + cache.getListenable().addListener(new ZkRegistryListener()); + } + + //Deserialize ServiceRecord from Zookeeper to populate AMRecord in cache + public static AMRecord getAMRecord(final ChildData childData) throws IOException { + byte[] data = childData.getData(); + // only the path appeared, there is no data yet + if (data.length == 0) { + return null; + } + String value = new String(data); + RegistryUtils.ServiceRecordMarshal marshal = new RegistryUtils.ServiceRecordMarshal(); + ServiceRecord serviceRecord = marshal.fromJson(value); + return new AMRecord(serviceRecord); + } + + @Override public AMRecord getRecord(String appId) { + if (amRecordCache.get(appId) == null) { + return null; + } + //Return a copy + return new AMRecord(amRecordCache.get(appId)); + } + + @Override public List getAllRecords() { + return amRecordCache.values().stream() + .map(record -> new AMRecord(record)).collect(Collectors.toList()); + } + + @Override public synchronized void addListener(AMRegistryClientListener listener) { + listeners.add(listener); + } + + //Callback for Zookeeper to update local cache + private class ZkRegistryListener implements PathChildrenCacheListener { + + @Override public void childEvent(final CuratorFramework client, final PathChildrenCacheEvent event) + throws Exception { + Preconditions.checkArgument(client != null && client.getState() == CuratorFrameworkState.STARTED, + "Curator client is not started"); + + ChildData childData = event.getData(); + switch (event.getType()) { + case CHILD_ADDED: + if(isEmpty(childData)) { + LOG.info("AppId allocated: {}", childData.getPath()); + } else { + AMRecord amRecord = getAMRecord(childData); + if (amRecord != null) { + LOG.info("AM registered with data: {}. Notifying {} listeners.", amRecord, listeners.size()); + amRecordCache.put(amRecord.getApplicationId().toString(), amRecord); + notifyOnAdded(amRecord); + } + } + break; + case CHILD_UPDATED: + if(isEmpty(childData)) { + throw new RuntimeException("AM updated with empty data"); + } else { + AMRecord amRecord = getAMRecord(childData); + if (amRecord != null) { + LOG.info("AM updated data: {}. Notifying {} listeners.", amRecord, listeners.size()); + amRecordCache.put(amRecord.getApplicationId().toString(), amRecord); + notifyOnAdded(amRecord); + } + } + break; + case CHILD_REMOVED: + if(isEmpty(childData)) { + LOG.info("Unused AppId unregistered: {}", childData.getPath()); + } else { + AMRecord amRecord = getAMRecord(childData); + if (amRecord != null) { + LOG.info("AM removed: {}. Notifying {} listeners.", amRecord, listeners.size()); + amRecordCache.remove(amRecord.getApplicationId().toString(), amRecord); + notifyOnRemoved(amRecord); + } + } + break; + default: + if(childData == null) { + LOG.info("Ignored event {}", event.getType()); + } else { + LOG.info("Ignored event {} for {}", event.getType(), childData.getPath()); + } + } + } + + private boolean isEmpty(ChildData childData) { + return childData == null || childData.getData() == null || childData.getData().length == 0; + } + } + + @Override + public void close() { + client.close(); + } +} \ No newline at end of file diff --git a/tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/ZkConfig.java b/tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/ZkConfig.java new file mode 100644 index 0000000000..38d32838b6 --- /dev/null +++ b/tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/ZkConfig.java @@ -0,0 +1,91 @@ +/** + * 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.tez.client.registry.zookeeper; + +import org.apache.curator.RetryPolicy; +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.CuratorFrameworkFactory; +import org.apache.curator.retry.ExponentialBackoffRetry; +import org.apache.hadoop.conf.Configuration; +import org.apache.tez.dag.api.TezConfiguration; + +import com.google.common.base.Preconditions; + +public class ZkConfig { + + private String zkQuorum; + private String zkNamespace; + private int curatorBackoffSleep; + private int curatorMaxRetries; + private int sessionTimeoutMs; + private int connectionTimeoutMs; + + public ZkConfig(Configuration conf) { + zkQuorum = conf.get(TezConfiguration.TEZ_AM_ZOOKEEPER_QUORUM); + Preconditions.checkNotNull(zkQuorum); + zkNamespace = conf.get(TezConfiguration.TEZ_AM_REGISTRY_NAMESPACE, + TezConfiguration.TEZ_AM_REGISTRY_NAMESPACE_DEFAULT); + Preconditions.checkNotNull(zkNamespace); + curatorBackoffSleep = conf.getInt(TezConfiguration.TEZ_AM_CURATOR_BACKOFF_SLEEP, + TezConfiguration.TEZ_AM_CURATOR_BACKOFF_SLEEP_DEFAULT); + curatorMaxRetries = conf.getInt(TezConfiguration.TEZ_AM_CURATOR_MAX_RETRIES, + TezConfiguration.TEZ_AM_CURATOR_MAX_RETRIES_DEFAULT); + sessionTimeoutMs = conf.getInt(TezConfiguration.TEZ_AM_CURATOR_SESSION_TIMEOUT, + TezConfiguration.TEZ_AM_CURATOR_SESSION_TIMEOUT_DEFAULT); + connectionTimeoutMs = conf.getInt(TezConfiguration.TEZ_AM_CURATOR_CONNECTION_TIMEOUT, + TezConfiguration.TEZ_AM_CURATOR_CONNECTION_TIMEOUT_DEFAULT); + } + + public String getZkQuorum() { + return zkQuorum; + } + + public String getZkNamespace() { + return zkNamespace; + } + + public int getCuratorBackoffSleep() { + return curatorBackoffSleep; + } + + public int getCuratorMaxRetries() { + return curatorMaxRetries; + } + + public int getSessionTimeoutMs() { + return sessionTimeoutMs; + } + + public int getConnectionTimeoutMs() { + return connectionTimeoutMs; + } + + public RetryPolicy getRetryPolicy() { + return new ExponentialBackoffRetry(getCuratorBackoffSleep(), getCuratorMaxRetries()); + } + + public CuratorFramework createCuratorFramework() { + return CuratorFrameworkFactory.newClient( + getZkQuorum(), + getSessionTimeoutMs(), + getConnectionTimeoutMs(), + getRetryPolicy() + ); + } +} diff --git a/tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/ZkFrameworkClient.java b/tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/ZkFrameworkClient.java new file mode 100644 index 0000000000..bf133c67c0 --- /dev/null +++ b/tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/ZkFrameworkClient.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.tez.client.registry.zookeeper; + +import java.io.IOException; + +import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ApplicationReport; +import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; +import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; +import org.apache.hadoop.yarn.api.records.YarnApplicationState; +import org.apache.hadoop.yarn.client.api.YarnClientApplication; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.util.Records; +import org.apache.tez.client.FrameworkClient; +import org.apache.tez.client.registry.AMRecord; +import org.apache.tez.dag.api.TezConfiguration; +import org.apache.tez.dag.api.TezException; + +public class ZkFrameworkClient extends FrameworkClient { + + private AMRecord amRecord; + private TezConfiguration tezConf; + private ZkAMRegistryClient amRegistryClient = null; + private boolean isRunning = false; + private String amHost; + private int amPort; + + @Override + public synchronized void init(TezConfiguration tezConf) { + this.tezConf = tezConf; + if (this.amRegistryClient == null) { + try { + this.amRegistryClient = ZkAMRegistryClient.getClient(tezConf); + this.isRunning = true; + } catch (Exception e) { + throw new RuntimeException(e); + } + } + } + + @Override public void start() { + try { + this.amRegistryClient.start(); + } catch (Exception e) { + throw new RuntimeException(e); + } + isRunning = true; + } + + @Override public void stop() { + isRunning = false; + amRegistryClient.close(); + } + + @Override public void close() throws IOException { + amRegistryClient.close(); + } + + @Override public YarnClientApplication createApplication() throws YarnException, IOException { + ApplicationSubmissionContext context = Records.newRecord(ApplicationSubmissionContext.class); + ApplicationId appId = amRecord.getApplicationId(); + context.setApplicationId(appId); + GetNewApplicationResponse response = Records.newRecord(GetNewApplicationResponse.class); + response.setApplicationId(appId); + return new YarnClientApplication(response, context); + } + + @Override public ApplicationId submitApplication(ApplicationSubmissionContext appSubmissionContext) + throws YarnException, IOException, TezException { + //Unused + return null; + } + + @Override public void killApplication(ApplicationId appId) throws YarnException, IOException { + amRegistryClient.close(); + } + + @Override public ApplicationReport getApplicationReport(ApplicationId appId) throws YarnException, IOException { + ApplicationReport report = Records.newRecord(ApplicationReport.class); + report.setApplicationId(appId); + report.setTrackingUrl(""); + amRecord = amRegistryClient.getRecord(appId.toString()); + // this could happen if the AM died, the AM record store under path will not exist + if (amRecord == null) { + report.setYarnApplicationState(YarnApplicationState.FINISHED); + report.setFinalApplicationStatus(FinalApplicationStatus.FAILED); + report.setDiagnostics("AM record not found (likely died) in zookeeper for application id: " + appId); + } else { + report.setHost(amRecord.getHost()); + amHost = amRecord.getHost(); + amPort = amRecord.getPort(); + report.setRpcPort(amRecord.getPort()); + report.setYarnApplicationState(YarnApplicationState.RUNNING); + } + return report; + } + + @Override public boolean isRunning() throws IOException { + return isRunning; + } + + @Override + public String getAmHost() { + return amHost; + } + + @Override + public int getAmPort() { + return amPort; + } +} diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java index 2176141d06..6caed299a7 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java @@ -866,6 +866,13 @@ public TezConfiguration(boolean loadDefaults) { public static final boolean TEZ_AM_NODE_UNHEALTHY_RESCHEDULE_TASKS_DEFAULT = false; + /** Int value. Port used for AM RPC*/ + @ConfigurationScope(Scope.AM) + @ConfigurationProperty(type="integer") + public static final String TEZ_AM_RPC_PORT = + TEZ_AM_PREFIX + "rpc.port"; + public static final int TEZ_AM_RPC_PORT_DEFAULT = 0; + /** Int value. Number of threads to handle client RPC requests. Expert level setting.*/ @ConfigurationScope(Scope.AM) @ConfigurationProperty(type="integer") @@ -2344,16 +2351,59 @@ static Set getPropertySet() { public static final String TEZ_TASK_ATTEMPT_HOOKS = TEZ_TASK_PREFIX + "attempt.hooks"; /** - * Comma-separated list of additional hadoop config files to load from CLASSPATH in standalone mode. + * String value */ @ConfigurationScope(Scope.AM) @ConfigurationProperty - public static final String TEZ_AM_STANDALONE_CONFS = TEZ_AM_PREFIX + "standalone.confs"; + public static final String TEZ_AM_ZOOKEEPER_QUORUM = TEZ_AM_PREFIX + "zookeeper.quorum"; + + /** + * String value + */ + @ConfigurationScope(Scope.AM) + @ConfigurationProperty + public static final String TEZ_AM_REGISTRY_NAMESPACE = TEZ_AM_PREFIX + "registry.namespace"; + public static final String TEZ_AM_REGISTRY_NAMESPACE_DEFAULT = "/tez_am/server"; + + /** + * Integer value + */ + @ConfigurationScope(Scope.AM) + @ConfigurationProperty + public static final String TEZ_AM_CURATOR_BACKOFF_SLEEP = TEZ_AM_PREFIX + "curator.backoff.sleep"; + public static final int TEZ_AM_CURATOR_BACKOFF_SLEEP_DEFAULT = 1000; + + /** + * Integer value + */ + @ConfigurationScope(Scope.AM) + @ConfigurationProperty + public static final String TEZ_AM_CURATOR_MAX_RETRIES = TEZ_AM_PREFIX + "curator.max.retries"; + public static final int TEZ_AM_CURATOR_MAX_RETRIES_DEFAULT = 3; + + /** + * Integer value (milliseconds) + */ + @ConfigurationScope(Scope.AM) + @ConfigurationProperty + public static final String TEZ_AM_CURATOR_SESSION_TIMEOUT = TEZ_AM_PREFIX + "curator.session.timeout"; + public static final int TEZ_AM_CURATOR_SESSION_TIMEOUT_DEFAULT = 60000; /** - * String value. The class to be used for the AM registry. + * Integer value (milliseconds) */ @ConfigurationScope(Scope.AM) @ConfigurationProperty - public static final String TEZ_AM_REGISTRY_CLASS = TEZ_AM_PREFIX + "registry.class"; + public static final String TEZ_AM_CURATOR_CONNECTION_TIMEOUT = TEZ_AM_PREFIX + "curator.connection.timeout"; + public static final int TEZ_AM_CURATOR_CONNECTION_TIMEOUT_DEFAULT = 15000; + + @ConfigurationProperty + public static final String TEZ_FRAMEWORK_MODE = TEZ_PREFIX + ".framework.mode"; + + /** + * List of additional hadoop config files to load from CLASSPATH in ZOOKEEPER_STANDALONE framework mode + */ + @ConfigurationScope(Scope.AM) + @ConfigurationProperty + public static final String TEZ_AM_STANDALONE_CONFS = TEZ_AM_PREFIX + "standalone.confs"; } diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConstants.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConstants.java index 71aff74801..5bb50e6c67 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConstants.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConstants.java @@ -29,24 +29,24 @@ public final class TezConstants { public static final String TEZ_APPLICATION_MASTER_CLASS = "org.apache.tez.dag.app.DAGAppMaster"; - + /** * Command-line argument to be set when running the Tez AM in session mode. */ public static final String TEZ_SESSION_MODE_CLI_OPTION = "session"; public static final String TEZ_TAR_LR_NAME = "tezlib"; - + /* * Tez AM Service Authorization * These are the same as MR which allows Tez to run in secure * mode without configuring service ACLs */ - public static final String - TEZ_AM_SECURITY_SERVICE_AUTHORIZATION_TASK_UMBILICAL = + public static final String + TEZ_AM_SECURITY_SERVICE_AUTHORIZATION_TASK_UMBILICAL = "security.job.task.protocol.acl"; - public static final String - TEZ_AM_SECURITY_SERVICE_AUTHORIZATION_CLIENT = + public static final String + TEZ_AM_SECURITY_SERVICE_AUTHORIZATION_CLIENT = "security.job.client.protocol.acl"; public static final String SERVICE_PLUGINS_DESCRIPTOR_JSON = "service_plugins_descriptor.json"; @@ -67,8 +67,8 @@ public final class TezConstants { public static final String TEZ_CONTAINER_LOG_PARAMS_SEPARATOR = ";"; public static final String TEZ_AM_LOCAL_RESOURCES_PB_FILE_NAME = - TezConfiguration.TEZ_SESSION_PREFIX + "local-resources.pb"; - + TezConfiguration.TEZ_SESSION_PREFIX + "local-resources.pb"; + public static final String TEZ_APPLICATION_TYPE = "TEZ"; /** * The service id for the NodeManager plugin used to share intermediate data @@ -84,7 +84,7 @@ public final class TezConstants { // Configuration keys used internally and not set by the users - + // These are session specific DAG ACL's. Currently here because these can only be specified // via code in the API. /** @@ -107,6 +107,8 @@ public final class TezConstants { private static final String TEZ_AM_SERVICE_PLUGIN_NAME_YARN_CONTAINERS = "TezYarn"; private static final String TEZ_AM_SERVICE_PLUGIN_NAME_IN_AM = "TezUber"; + public static final String TEZ_AM_EXTERNAL_ID = "TEZ_AM_EXTERNAL_ID"; + public static final String TEZ_FRAMEWORK_MODE = "TEZ_FRAMEWORK_MODE"; public static String getTezYarnServicePluginName() { return TEZ_AM_SERVICE_PLUGIN_NAME_YARN_CONTAINERS; diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java index 827d333371..f337e944db 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java @@ -286,7 +286,6 @@ boolean createAMProxyIfNeeded() throws IOException, TezException, // attempt not running return false; } - proxy = TezClientUtils.getAMProxy(conf, appReport.getHost(), appReport.getRpcPort(), appReport.getClientToAMToken(), ugi); return true; diff --git a/tez-api/src/main/java/org/apache/tez/frameworkplugins/ClientFrameworkService.java b/tez-api/src/main/java/org/apache/tez/frameworkplugins/ClientFrameworkService.java new file mode 100644 index 0000000000..adbf66323d --- /dev/null +++ b/tez-api/src/main/java/org/apache/tez/frameworkplugins/ClientFrameworkService.java @@ -0,0 +1,35 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.tez.frameworkplugins; + +import java.util.Optional; + +import org.apache.hadoop.conf.Configuration; +import org.apache.tez.client.FrameworkClient; +import org.apache.tez.client.registry.AMRegistryClient; + +/* + FrameworkService that runs code within the client process that is using TezClient + Bundles together a compatible FrameworkClient and AMRegistryClient + */ +public interface ClientFrameworkService extends FrameworkService { + //Provide an impl. for org.apache.tez.client.FrameworkClient + Optional createOrGetFrameworkClient(Configuration conf); + //Provide an impl. for org.apache.tez.registry.AMRegistryClient + Optional createOrGetRegistryClient(Configuration conf); +} diff --git a/tez-api/src/main/java/org/apache/tez/frameworkplugins/FrameworkMode.java b/tez-api/src/main/java/org/apache/tez/frameworkplugins/FrameworkMode.java new file mode 100644 index 0000000000..b7d6fe1a39 --- /dev/null +++ b/tez-api/src/main/java/org/apache/tez/frameworkplugins/FrameworkMode.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.tez.frameworkplugins; + +/* + A FrameworkMode is a pair of classes implementing (ClientFrameworkService, ServerFrameworkService) + Clients using one FrameworkMode should only connect to AMs using the same FrameworkMode + It is the responsibility of the user to setup their environment/configs to ensure matching modes + e.g. a client using a mode that requires a Zookeeper-based registry should not be configured + to interact with AMs that do not keep a Zookeeper-based registry + */ +public enum FrameworkMode { + + STANDALONE_ZOOKEEPER( + "org.apache.tez.frameworkplugins.zookeeper.ZookeeperStandaloneClientFrameworkService", + "org.apache.tez.frameworkplugins.zookeeper.ZookeeperStandaloneServerFrameworkService"); + + String clientClassName; + String serverClassName; + + FrameworkMode(String clientClassName, String serverClassName) { + this.clientClassName = clientClassName; + this.serverClassName = serverClassName; + } +} diff --git a/tez-api/src/main/java/org/apache/tez/frameworkplugins/FrameworkService.java b/tez-api/src/main/java/org/apache/tez/frameworkplugins/FrameworkService.java new file mode 100644 index 0000000000..aff52b55a2 --- /dev/null +++ b/tez-api/src/main/java/org/apache/tez/frameworkplugins/FrameworkService.java @@ -0,0 +1,24 @@ +/** + * 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.tez.frameworkplugins; + +//Marker for a FrameworkService +//extended by ClientFrameworkService and ServerFrameworkService +public interface FrameworkService { + +} diff --git a/tez-api/src/main/java/org/apache/tez/frameworkplugins/FrameworkUtils.java b/tez-api/src/main/java/org/apache/tez/frameworkplugins/FrameworkUtils.java new file mode 100644 index 0000000000..ebf6e9b3ea --- /dev/null +++ b/tez-api/src/main/java/org/apache/tez/frameworkplugins/FrameworkUtils.java @@ -0,0 +1,98 @@ +/** + * 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.tez.frameworkplugins; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Optional; +import java.util.ServiceLoader; + +import javax.annotation.Nullable; + +import org.apache.tez.common.ReflectionUtils; +import org.apache.tez.dag.api.TezConfiguration; +import org.apache.tez.dag.api.TezConstants; +import org.apache.tez.dag.api.TezReflectionException; + +public class FrameworkUtils { + + /* + Searches for a FrameworkService provider which implements a target interface. + The interface should be either ClientFrameworkService or ServerFrameworkService. + Depending on which interface is used, either the client or server class of a + matching FrameworkMode will be used as the implementation. + + NOTE: Layering of FrameworkServices in a decorator-style is currently not supported + + An implementation is searched in the following order: + 1. If conf is not null and the parameter TEZ_FRAMEWORK_MODE is set: + the value of TEZ_FRAMEWORK_MODE from the conf will be used + 2. If conf is null or the parameter TEZ_FRAMEWORK_MODE is not set + and the environment var TEZ_FRAMEWORK_MODE is not empty: + the value of the environment var will be used + 3. Otherwise: + the default java.util.ServiceLoader behavior will be used, + i.e. the implementation classname should appear in a file on the classpath at the location + META-INF/services/org.apache.tez.frameworkplugins.ClientFrameworkService + or META-INF/services/org.apache.tez.frameworkplugins.ServerFrameworkService + */ + public static Optional get(Class interfaze, @Nullable TezConfiguration conf) { + try { + if ((conf != null) && (conf.get(TezConfiguration.TEZ_FRAMEWORK_MODE) != null)) { + return getByMode(interfaze, conf.get(TezConfiguration.TEZ_FRAMEWORK_MODE)); + } else if (System.getenv(TezConstants.TEZ_FRAMEWORK_MODE) != null) { + return getByMode(interfaze, System.getenv(TezConstants.TEZ_FRAMEWORK_MODE)); + } else { + return getByServiceLoader(interfaze); + } + } catch(TezReflectionException e) { + throw new RuntimeException("Failed to load framework service for interface: " + interfaze.getName()); + } + } + + private static Optional getByServiceLoader(Class interfaze) { + List services = new ArrayList<>(); + ServiceLoader frameworkService = ServiceLoader.load(interfaze); + Iterator it = frameworkService.iterator(); + while (it.hasNext()) { + T service = it.next(); + services.add(service); + } + if(services.size() == 0) { + return Optional.empty(); + } else if (services.size() > 1) { + throw new RuntimeException("Layering of multiple framework services is not supported." + + " Please provide only one implementation class in configuration."); + } + //services is guaranteed to have one element at this point + return Optional.of(services.get(0)); + } + + private static Optional getByMode(Class interfaze, String mode) throws TezReflectionException { + mode = mode.toUpperCase(); + String clazz = null; + if(interfaze == ClientFrameworkService.class) { + clazz = FrameworkMode.valueOf(mode).clientClassName; + } else { + clazz = FrameworkMode.valueOf(mode).serverClassName; + } + return Optional.of(ReflectionUtils.createClazzInstance(clazz)); + } + +} diff --git a/tez-api/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZookeeperStandaloneClientFrameworkService.java b/tez-api/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZookeeperStandaloneClientFrameworkService.java new file mode 100644 index 0000000000..0d9f8c26c8 --- /dev/null +++ b/tez-api/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZookeeperStandaloneClientFrameworkService.java @@ -0,0 +1,26 @@ +package org.apache.tez.frameworkplugins.zookeeper; + +import java.util.Optional; + +import org.apache.hadoop.conf.Configuration; +import org.apache.tez.client.FrameworkClient; +import org.apache.tez.client.registry.AMRegistryClient; +import org.apache.tez.client.registry.zookeeper.ZkAMRegistryClient; +import org.apache.tez.client.registry.zookeeper.ZkFrameworkClient; +import org.apache.tez.frameworkplugins.ClientFrameworkService; + +public class ZookeeperStandaloneClientFrameworkService implements ClientFrameworkService { + @Override public Optional createOrGetFrameworkClient(Configuration conf) { + return Optional.of(new ZkFrameworkClient()); + } + + @Override public Optional createOrGetRegistryClient(Configuration conf) { + ZkAMRegistryClient registry = ZkAMRegistryClient.getClient(conf); + try { + registry.start(); + } catch (Exception e) { + throw new RuntimeException(e); + } + return Optional.of(registry); + } +} diff --git a/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java b/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java index 46a972c1d6..1af958514a 100644 --- a/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java +++ b/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java @@ -221,6 +221,16 @@ public void testTezClientSession() throws Exception { testTezClient(true, true, "testTezClientSession"); } + @Test (timeout = 5000) + public void testTezclientReconnect() throws Exception { + testTezClientReconnect(true); + } + + @Test (timeout = 5000, expected = IllegalStateException.class) + public void testTezclientReconnectNoSession() throws Exception { + testTezClientReconnect(false); + } + @Test (timeout = 5000) public void testTezClientSessionLargeDAGPlan() throws Exception { // request size is within threshold of being serialized @@ -387,18 +397,18 @@ public TezClientForTest testTezClient(boolean isSession, boolean shouldStop, Str assertTrue(context.getAMContainerSpec().getLocalResources().containsKey( lrName1)); } - + // add resources String lrName2 = "LR2"; lrs.clear(); lrs.put(lrName2, LocalResource.newInstance(URL.newInstance("file", "localhost", 0, "/test2"), LocalResourceType.FILE, LocalResourceVisibility.PUBLIC, 1, 1)); client.addAppMasterLocalFiles(lrs); - + ApplicationId appId2 = ApplicationId.newInstance(0, 2); when(client.mockYarnClient.createApplication().getNewApplicationResponse().getApplicationId()) .thenReturn(appId2); - + when(client.mockYarnClient.getApplicationReport(appId2).getYarnApplicationState()) .thenReturn(YarnApplicationState.RUNNING); dag = DAG.create("DAG-2-" + dagName).addVertex( @@ -447,6 +457,93 @@ public TezClientForTest testTezClient(boolean isSession, boolean shouldStop, Str return client; } + public void testTezClientReconnect(boolean isSession) throws Exception { + //Setup 1 + Map lrs = Maps.newHashMap(); + String lrName1 = "LR1"; + lrs.put(lrName1, LocalResource.newInstance(URL.newInstance("file", "localhost", 0, "/test"), + LocalResourceType.FILE, LocalResourceVisibility.PUBLIC, 1, 1)); + + //Client 1 + TezClientForTest client = configureAndCreateTezClient(lrs, isSession, null); + + //Submission Context 1 + ArgumentCaptor captor = ArgumentCaptor.forClass(ApplicationSubmissionContext.class); + when(client.mockYarnClient.getApplicationReport(client.mockAppId).getYarnApplicationState()) + .thenReturn(YarnApplicationState.RUNNING); + + //Client 1 start + client.start(); + + //Client 1 verify + verify(client.mockYarnClient, times(1)).init((Configuration)any()); + verify(client.mockYarnClient, times(1)).start(); + + if (isSession) { + verify(client.mockYarnClient, times(1)).submitApplication(captor.capture()); + ApplicationSubmissionContext context = captor.getValue(); + Assert.assertEquals(3, context.getAMContainerSpec().getLocalResources().size()); + assertTrue(context.getAMContainerSpec().getLocalResources().containsKey( + TezConstants.TEZ_AM_LOCAL_RESOURCES_PB_FILE_NAME)); + assertTrue(context.getAMContainerSpec().getLocalResources().containsKey( + TezConstants.TEZ_PB_BINARY_CONF_NAME)); + assertTrue(context.getAMContainerSpec().getLocalResources().containsKey( + lrName1)); + } else { + verify(client.mockYarnClient, times(0)).submitApplication(captor.capture()); + } + + //DAG 1 resources + Map lrDAG = Collections.singletonMap(lrName1, LocalResource + .newInstance(URL.newInstance("file", "localhost", 0, "/test1"), LocalResourceType.FILE, + LocalResourceVisibility.PUBLIC, 1, 1)); + + //DAG 1 setup + Vertex vertex = Vertex.create("Vertex", ProcessorDescriptor.create("P"), 1, + Resource.newInstance(1, 1)); + DAG dag = DAG.create("DAG").addVertex(vertex).addTaskLocalFiles(lrDAG); + + //DAG 1 submit + DAGClient dagClient = client.submitDAG(dag); + + //DAG 1 assertions + assertTrue(dagClient.getExecutionContext().contains(client.mockAppId.toString())); + assertEquals(dagClient.getSessionIdentifierString(), client.mockAppId.toString()); + + //Client 2 reuse appId + ApplicationId appId = client.getAppMasterApplicationId(); + + //Client 2 reuse lrs + TezClientForTest client2 = configureAndCreateTezClient(lrs, isSession, null); + + //Submission Context 2 + ArgumentCaptor captorClient2 = ArgumentCaptor.forClass(ApplicationSubmissionContext.class); + when(client2.mockYarnClient.getApplicationReport(client2.mockAppId).getYarnApplicationState()) + .thenReturn(YarnApplicationState.RUNNING); + + //Client 2 reconnect + client2.getClient(appId); + assertEquals(client2.mockAppId, appId); + + //Client 2 verify + verify(client2.mockYarnClient, times(1)).init((Configuration)any()); + verify(client2.mockYarnClient, times(1)).start(); + //New AM should not be submitted + verify(client2.mockYarnClient, times(0)).submitApplication(captorClient2.capture()); + + //DAG 2 setup + Vertex vertex2 = Vertex.create("Vertex2", ProcessorDescriptor.create("P"), 1, + Resource.newInstance(1, 1)); + dag = DAG.create("DAG2").addVertex(vertex2).addTaskLocalFiles(lrDAG); + + //DAG 2 submit + dagClient = client2.submitDAG(dag); + + //DAG 2 assertions + assertTrue(dagClient.getExecutionContext().contains(appId.toString())); + assertEquals(dagClient.getSessionIdentifierString(), appId.toString()); + } + @Test (timeout=5000) public void testPreWarm() throws Exception { TezClientForTest client = configureAndCreateTezClient(); @@ -757,7 +854,6 @@ public void testClientBuilder() { // Session mode via conf tezClient = TezClient.newBuilder("client", tezConfWitSession).build(); - assertTrue(tezClient.isSession); assertNull(tezClient.servicePluginsDescriptor); assertNotNull(tezClient.apiVersionInfo); amConf = tezClient.amConfig; @@ -769,7 +865,6 @@ public void testClientBuilder() { // Non-Session mode via conf tezClient = TezClient.newBuilder("client", tezConfNoSession).build(); - assertFalse(tezClient.isSession); assertNull(tezClient.servicePluginsDescriptor); assertNotNull(tezClient.apiVersionInfo); amConf = tezClient.amConfig; @@ -781,7 +876,6 @@ public void testClientBuilder() { // no-session via config. API explicit session. tezClient = TezClient.newBuilder("client", tezConfNoSession).setIsSession(true).build(); - assertTrue(tezClient.isSession); assertNull(tezClient.servicePluginsDescriptor); assertNotNull(tezClient.apiVersionInfo); amConf = tezClient.amConfig; @@ -795,7 +889,6 @@ public void testClientBuilder() { tezClient = TezClient.newBuilder("client", tezConfWitSession).setCredentials(credentials) .setLocalResources(localResourceMap).setServicePluginDescriptor(servicePluginsDescriptor) .build(); - assertTrue(tezClient.isSession); assertEquals(servicePluginsDescriptor, tezClient.servicePluginsDescriptor); assertNotNull(tezClient.apiVersionInfo); amConf = tezClient.amConfig; diff --git a/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java b/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java index 03818060c6..43acde2965 100644 --- a/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java +++ b/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java @@ -697,6 +697,24 @@ public void testDefaultLoggingJavaOptsWithRootLogger() { Assert.assertTrue(javaOpts.contains("-DtestProperty=value")); } + @Test (timeout = 5000) + public void testConfYarnZkWorkaround() { + Configuration conf = new Configuration(false); + String val = "localhost:2181"; + conf.set("yarn.resourcemanager.zk-address", val); + + Map expected = new HashMap(); + expected.put("yarn.resourcemanager.zk-address", val); + + ConfigurationProto confProto = TezClientUtils.createFinalConfProtoForApp(conf, null); + + for (PlanKeyValuePair kvPair : confProto.getConfKeyValuesList()) { + String v = expected.remove(kvPair.getKey()); + assertEquals(v, kvPair.getValue()); + } + assertTrue(expected.isEmpty()); + } + @Test (timeout = 5000) public void testConfSerializationForAm() { Configuration conf =new Configuration(false); diff --git a/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java b/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java index 661d77320e..3f93de92d9 100644 --- a/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java +++ b/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java @@ -46,8 +46,10 @@ import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.log4j.Appender; import org.apache.log4j.PatternLayout; +import org.apache.tez.client.TezClientUtils; import org.apache.tez.common.io.NonSyncByteArrayOutputStream; import org.apache.tez.dag.api.DagTypeConverters; +import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.dag.api.TezConstants; import org.apache.tez.dag.api.TezUncheckedException; import org.apache.tez.dag.api.records.DAGProtos; @@ -58,6 +60,7 @@ import org.apache.tez.dag.records.TezTaskAttemptID; import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.hadoop.shim.HadoopShim; +import org.apache.tez.serviceplugins.api.ServicePluginsDescriptor; import org.apache.tez.serviceplugins.api.TaskAttemptEndReason; import org.apache.tez.util.StopWatch; @@ -90,6 +93,21 @@ public static Configuration readTezConfigurationXml(InputStream is) { return configuration; } + public static ConfigurationProto loadConfProtoFromText() throws IOException { + try(InputStream cis = ClassLoader.getSystemResourceAsStream(TezConfiguration.TEZ_SITE_XML); + InputStream sis = ClassLoader.getSystemResourceAsStream(TezConstants.SERVICE_PLUGINS_DESCRIPTOR_JSON)) { + Configuration confFromXml = TezUtilsInternal.readTezConfigurationXml(cis); + for(String confFile : confFromXml.getTrimmedStringCollection(TezConfiguration.TEZ_AM_STANDALONE_CONFS)) { + try(InputStream additionalInput = ClassLoader.getSystemResourceAsStream(confFile)) { + Configuration additionalConfFromXml = TezUtilsInternal.readTezConfigurationXml(additionalInput); + confFromXml.addResource(additionalConfFromXml); + } + } + ServicePluginsDescriptor pluginsDescriptor = TezClientUtils.createPluginsDescriptorFromJSON(sis); + return TezClientUtils.createFinalConfProtoForApp(confFromXml, pluginsDescriptor); + } + } + public static void addUserSpecifiedTezConfiguration(Configuration conf, List kvPairList) { if (kvPairList != null && !kvPairList.isEmpty()) { diff --git a/tez-dag/pom.xml b/tez-dag/pom.xml index a765fb708a..37ea5432d7 100644 --- a/tez-dag/pom.xml +++ b/tez-dag/pom.xml @@ -44,6 +44,10 @@ org.apache.tez tez-api + + org.apache.tez + tez-mapreduce + org.apache.tez tez-common @@ -83,6 +87,10 @@ org.apache.hadoop hadoop-yarn-client + + org.apache.hadoop + hadoop-registry + org.apache.hadoop hadoop-yarn-server-web-proxy @@ -148,6 +156,12 @@ test-jar test + + org.apache.curator + curator-test + ${curator.version} + test + diff --git a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java index 79f9f15a64..9b65f2f452 100644 --- a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java +++ b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java @@ -415,7 +415,7 @@ protected DAGAppMaster createDAGAppMaster(ApplicationAttemptId applicationAttemp versionInfo.getVersion(), credentials, jobUserName, amPluginDescriptorProto) : new DAGAppMaster(applicationAttemptId, cId, currentHost, nmPort, nmHttpPort, SystemClock.getInstance(), appSubmitTime, isSession, userDir, localDirs, logDirs, - versionInfo.getVersion(), credentials, jobUserName, amPluginDescriptorProto, null); + versionInfo.getVersion(), credentials, jobUserName, amPluginDescriptorProto); } @Override diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientServer.java b/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientServer.java index 20d1563e4e..1c62b0789a 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientServer.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientServer.java @@ -52,6 +52,7 @@ public class DAGClientServer extends AbstractService { DAGClientHandler realInstance; Server server; InetSocketAddress bindAddress; + ApplicationAttemptId applicationAttemptId; final FileSystem stagingFs; public DAGClientServer(DAGClientHandler realInstance, @@ -59,6 +60,7 @@ public DAGClientServer(DAGClientHandler realInstance, super("DAGClientRPCServer"); this.realInstance = realInstance; this.secretManager = new ClientToAMTokenSecretManager(attemptId, null); + this.applicationAttemptId = attemptId; this.stagingFs = stagingFs; } @@ -66,7 +68,9 @@ public DAGClientServer(DAGClientHandler realInstance, public void serviceStart() { try { Configuration conf = getConfig(); - InetSocketAddress addr = new InetSocketAddress(0); + int rpcPort = conf.getInt(TezConfiguration.TEZ_AM_RPC_PORT, + TezConfiguration.TEZ_AM_RPC_PORT_DEFAULT); + InetSocketAddress addr = new InetSocketAddress(rpcPort); DAGClientAMProtocolBlockingPBServerImpl service = new DAGClientAMProtocolBlockingPBServerImpl(realInstance, stagingFs); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/client/registry/zookeeper/ZkAMRegistry.java b/tez-dag/src/main/java/org/apache/tez/dag/api/client/registry/zookeeper/ZkAMRegistry.java new file mode 100644 index 0000000000..c0daecef38 --- /dev/null +++ b/tez-dag/src/main/java/org/apache/tez/dag/api/client/registry/zookeeper/ZkAMRegistry.java @@ -0,0 +1,168 @@ +/** + * 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.tez.dag.api.client.registry.zookeeper; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Optional; + +import org.apache.curator.RetryLoop; +import org.apache.curator.RetryPolicy; +import org.apache.curator.framework.CuratorFramework; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.registry.client.binding.RegistryUtils; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.tez.client.registry.AMRecord; +import org.apache.tez.client.registry.AMRegistry; +import org.apache.tez.client.registry.zookeeper.ZkConfig; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.data.Stat; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Curator/Zookeeper impl of AMRegistry (for internal use only) + * Clients should use org.apache.tez.dag.api.client.registry.zookeeper.ZkAMRegistryClient instead + */ +@InterfaceAudience.Private +public class ZkAMRegistry extends AMRegistry { + + private static final Logger LOG = LoggerFactory.getLogger(ZkAMRegistry.class); + + private CuratorFramework client = null; + private String namespace = null; + private List amRecords = new ArrayList<>(); + private ZkConfig zkConfig = null; + private boolean started = false; + private String externalId; + + public ZkAMRegistry(String externalId) { + super("ZkAMRegistry"); + this.externalId = externalId; + } + + @Override + public void serviceInit(Configuration conf) { + if(zkConfig == null) { + zkConfig = new ZkConfig(conf); + this.client = zkConfig.createCuratorFramework(); + this.namespace = zkConfig.getZkNamespace(); + LOG.info("AMRegistryZkImpl initialized"); + } + } + + @Override public void serviceStart() throws Exception { + if(!started) { + client.start(); + started = true; + LOG.info("AMRegistryZkImpl started"); + } + } + + //Deletes from Zookeeper AMRecords that were added by this instance + @Override public void serviceStop() throws Exception { + List records = new ArrayList<>(amRecords); + for(AMRecord amRecord : records) { + remove(amRecord); + } + client.close(); + LOG.info("AMRegistryZkImpl shutdown"); + } + + //Serialize AMRecord to ServiceRecord and deliver the JSON bytes to + //zkNode at the path: / + @Override public void add(AMRecord server) throws Exception { + RegistryUtils.ServiceRecordMarshal marshal = new RegistryUtils.ServiceRecordMarshal(); + String json = marshal.toJson(server.toServiceRecord()); + try { + final String path = namespace + "/" + server.getApplicationId().toString(); + client.setData().forPath(path, json.getBytes()); + LOG.info("Added AMRecord to zkpath {}", path); + } catch(KeeperException.NoNodeException nne) { + client.create().creatingParentContainersIfNeeded().withMode(CreateMode.EPHEMERAL).forPath(namespace + "/" + server.getApplicationId().toString(), json.getBytes()); + } + amRecords.add(server); + } + + @Override public void remove(AMRecord server) throws Exception { + amRecords.remove(server); + final String path = namespace + "/" + server.getApplicationId().toString(); + client.delete().forPath(path); + LOG.info("Deleted AMRecord from zkpath {}", path); + } + + @Override + public Optional generateNewId() throws Exception { + createNamespaceIfNotExists(); + long namespaceCreationTime = getNamespaceCreationTime(); + + boolean success = false; + long startTime = System.currentTimeMillis(); + RetryPolicy retryPolicy = zkConfig.getRetryPolicy(); + int tryId = 0; + for(int i = 0; (i < zkConfig.getCuratorMaxRetries()) && !success; i++) { + List children = client.getChildren().forPath(namespace); + if((children != null) && (children.size() != 0)) { + Collections.sort(children, Collections.reverseOrder()); + String last = children.get(0); + ApplicationId lastAppId = ApplicationId.fromString(last); + tryId = lastAppId.getId() + 1; + } + ApplicationId tryAppId = ApplicationId.newInstance(namespaceCreationTime, tryId); + try { + client + .create() + .withMode(CreateMode.EPHEMERAL) + .forPath(namespace + "/" + tryAppId.toString(), new byte[0]); + success = true; + } catch(KeeperException.NodeExistsException nodeExists) { + long elapsedTime = System.currentTimeMillis() - startTime; + retryPolicy.allowRetry(i + 1, elapsedTime, RetryLoop.getDefaultRetrySleeper()); + tryId++; + } + } + if(success) { + return Optional.of(ApplicationId.newInstance(namespaceCreationTime, tryId)); + } else { + throw new RuntimeException("Could not obtain unique ApplicationId after " + + zkConfig.getCuratorMaxRetries() + " tries"); + } + } + + @Override public AMRecord createAmRecord(ApplicationId appId, String hostName, int port) { + return new AMRecord(appId, hostName, port, externalId); + } + + private long getNamespaceCreationTime() throws Exception { + Stat stat = client.checkExists().forPath(namespace); + return stat.getCtime(); + } + + private void createNamespaceIfNotExists() throws Exception { + try { + client.create().creatingParentContainersIfNeeded().forPath(namespace); + } catch(KeeperException.NodeExistsException nodeExists) { + LOG.info("Namespace already exists, will use existing: {}", namespace); + } + } +} diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java index ec4a89be03..1be36212c0 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java @@ -20,11 +20,13 @@ -import java.io.BufferedInputStream; +import java.io.BufferedReader; +import java.io.ByteArrayInputStream; import java.io.File; import java.io.FileInputStream; import java.io.FileNotFoundException; import java.io.IOException; +import java.io.InputStreamReader; import java.io.PrintWriter; import java.net.URI; import java.net.URISyntaxException; @@ -46,6 +48,7 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Objects; +import java.util.Optional; import java.util.Random; import java.util.Set; import java.util.Timer; @@ -78,7 +81,6 @@ import org.apache.hadoop.util.ShutdownHookManager; import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler; import org.apache.hadoop.yarn.api.ApplicationConstants; -import org.apache.hadoop.yarn.api.ApplicationConstants.Environment; import org.apache.hadoop.yarn.api.records.ApplicationAccessType; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId; @@ -98,6 +100,7 @@ import org.apache.tez.client.CallerContext; import org.apache.tez.client.TezClientUtils; import org.apache.tez.client.registry.AMRecord; +import org.apache.tez.client.registry.AMRegistry; import org.apache.tez.common.AsyncDispatcher; import org.apache.tez.common.AsyncDispatcherConcurrent; import org.apache.tez.common.ContainerSignatureMatcher; @@ -126,10 +129,8 @@ import org.apache.tez.dag.api.UserPayload; import org.apache.tez.dag.api.client.DAGClientHandler; import org.apache.tez.dag.api.client.DAGClientServer; -import org.apache.tez.dag.api.client.registry.AMRegistry; import org.apache.tez.dag.api.records.DAGProtos; import org.apache.tez.dag.api.records.DAGProtos.AMPluginDescriptorProto; -import org.apache.tez.dag.api.records.DAGProtos.ConfigurationProto; import org.apache.tez.dag.api.records.DAGProtos.DAGPlan; import org.apache.tez.dag.api.records.DAGProtos.PlanLocalResourcesProto; import org.apache.tez.dag.api.records.DAGProtos.VertexPlan; @@ -183,9 +184,11 @@ import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.dag.records.TezTaskAttemptID; import org.apache.tez.dag.records.TezVertexID; -import org.apache.tez.dag.utils.AMRegistryUtils; import org.apache.tez.dag.utils.RelocalizationUtils; import org.apache.tez.dag.utils.Simple2LevelVersionComparator; +import org.apache.tez.frameworkplugins.AmExtensions; +import org.apache.tez.frameworkplugins.FrameworkUtils; +import org.apache.tez.frameworkplugins.ServerFrameworkService; import org.apache.tez.hadoop.shim.HadoopShim; import org.apache.tez.hadoop.shim.HadoopShimsLoader; import org.apache.tez.runtime.hook.TezDAGHook; @@ -225,6 +228,10 @@ public class DAGAppMaster extends AbstractService { private static final Logger LOG = LoggerFactory.getLogger(DAGAppMaster.class); + private static Optional frameworkService = + FrameworkUtils.get(ServerFrameworkService.class, null); + public static Optional amExts = + frameworkService.flatMap(fs -> fs.createOrGetDAGAppMasterExtensions()); /** * Priority of the DAGAppMaster shutdown hook. @@ -243,7 +250,6 @@ public class DAGAppMaster extends AbstractService { private String appName; private final ApplicationAttemptId appAttemptID; private final ContainerId containerID; - private String amUUID; private final String nmHost; private final int nmPort; private final int nmHttpPort; @@ -346,8 +352,7 @@ public DAGAppMaster(ApplicationAttemptId applicationAttemptId, ContainerId containerId, String nmHost, int nmPort, int nmHttpPort, Clock clock, long appSubmitTime, boolean isSession, String workingDirectory, String [] localDirs, String[] logDirs, String clientVersion, - Credentials credentials, String jobUserName, AMPluginDescriptorProto pluginDescriptorProto, - String amUUID) { + Credentials credentials, String jobUserName, AMPluginDescriptorProto pluginDescriptorProto) { super(DAGAppMaster.class.getName()); this.mdcContext = LoggingUtils.setupLog4j(); this.clock = clock; @@ -355,7 +360,6 @@ public DAGAppMaster(ApplicationAttemptId applicationAttemptId, this.appSubmitTime = appSubmitTime; this.appAttemptID = applicationAttemptId; this.containerID = containerId; - this.amUUID = amUUID; this.nmHost = nmHost; this.nmPort = nmPort; this.nmHttpPort = nmHttpPort; @@ -491,20 +495,23 @@ protected void serviceInit(final Configuration conf) throws Exception { addIfService(dispatcher, false); - recoveryDataDir = TezCommonUtils.getRecoveryPath(tezSystemStagingDir, conf); - recoveryFS = recoveryDataDir.getFileSystem(conf); - currentRecoveryDataDir = TezCommonUtils.getAttemptRecoveryPath(recoveryDataDir, - appAttemptID.getAttemptId()); - if (LOG.isDebugEnabled()) { - LOG.debug("Stage directory information for AppAttemptId :" + this.appAttemptID - + " tezSystemStagingDir :" + tezSystemStagingDir + " recoveryDataDir :" + recoveryDataDir - + " recoveryAttemptDir :" + currentRecoveryDataDir); - } + + recoveryDataDir = TezCommonUtils.getRecoveryPath(tezSystemStagingDir, conf); + recoveryFS = recoveryDataDir.getFileSystem(conf); + currentRecoveryDataDir = TezCommonUtils.getAttemptRecoveryPath(recoveryDataDir, + appAttemptID.getAttemptId()); + if (LOG.isDebugEnabled()) { + LOG.debug("Stage directory information for AppAttemptId :" + this.appAttemptID + + " tezSystemStagingDir :" + tezSystemStagingDir + " recoveryDataDir :" + recoveryDataDir + + " recoveryAttemptDir :" + currentRecoveryDataDir); + } recoveryEnabled = conf.getBoolean(TezConfiguration.DAG_RECOVERY_ENABLED, TezConfiguration.DAG_RECOVERY_ENABLED_DEFAULT); initClientRpcServer(); + + taskHeartbeatHandler = createTaskHeartbeatHandler(context, conf); addIfService(taskHeartbeatHandler, true); @@ -513,8 +520,10 @@ protected void serviceInit(final Configuration conf) throws Exception { jobTokenSecretManager = new JobTokenSecretManager(amConf); - sessionToken = - TokenCache.getSessionToken(amCredentials); + sessionToken = amExts.flatMap(amExtensions -> amExtensions.getSessionToken( + appAttemptID, jobTokenSecretManager, amCredentials + )).orElse(TokenCache.getSessionToken(amCredentials)); + if (sessionToken == null) { throw new RuntimeException("Could not find session token in AM Credentials"); } @@ -524,8 +533,6 @@ protected void serviceInit(final Configuration conf) throws Exception { jobTokenSecretManager.addTokenForJob( appAttemptID.getApplicationId().toString(), sessionToken); - - //service to handle requests to TaskUmbilicalProtocol taskCommunicatorManager = createTaskCommunicatorManager(context, taskHeartbeatHandler, containerHeartbeatHandler, taskCommunicatorDescriptors); @@ -602,15 +609,13 @@ protected void serviceInit(final Configuration conf) throws Exception { if (!versionMismatch) { if (isSession) { - try (BufferedInputStream sessionResourcesStream = - new BufferedInputStream( - new FileInputStream(new File(workingDirectory, - TezConstants.TEZ_AM_LOCAL_RESOURCES_PB_FILE_NAME)))) { - PlanLocalResourcesProto amLocalResourceProto = PlanLocalResourcesProto - .parseDelimitedFrom(sessionResourcesStream); - amResources.putAll(DagTypeConverters - .convertFromPlanLocalResources(amLocalResourceProto)); + DAGProtos.PlanLocalResourcesProto amLocalResourceProto = + amExts.flatMap(amExtensions -> amExtensions.getAdditionalSessionResources(workingDirectory)) + .orElse(null); + if(amLocalResourceProto == null) { + amLocalResourceProto = getAdditionalSessionResources(workingDirectory); } + amResources.putAll(DagTypeConverters.convertFromPlanLocalResources(amLocalResourceProto)); } } @@ -622,10 +627,12 @@ protected void serviceInit(final Configuration conf) throws Exception { Executors.newFixedThreadPool(threadCount, new ThreadFactoryBuilder() .setDaemon(true).setNameFormat("App Shared Pool - #%d").build()); execService = MoreExecutors.listeningDecorator(rawExecutor); - - AMRegistry amRegistry = AMRegistryUtils.createAMRegistry(conf); - initAmRegistry(appAttemptID.getApplicationId(), amUUID, amRegistry, clientRpcServer); - addIfService(amRegistry, false); + Optional amRegistry = + frameworkService.flatMap(service -> service.createOrGetAMRegistry(conf)); + if(amRegistry.isPresent()) { + initAmRegistry(appAttemptID.getApplicationId(), amRegistry.get(), clientRpcServer); + addIfService(amRegistry.get(), false); + } initServices(conf); super.serviceInit(conf); @@ -649,26 +656,39 @@ protected void serviceInit(final Configuration conf) throws Exception { } } - protected void initClientRpcServer() { - clientRpcServer = new DAGClientServer(clientHandler, appAttemptID, recoveryFS); - addIfService(clientRpcServer, true); + private static DAGProtos.PlanLocalResourcesProto getAdditionalSessionResources(String workingDirectory) throws IOException { + FileInputStream sessionResourcesStream = null; + try { + sessionResourcesStream = + new FileInputStream(new File(workingDirectory, TezConstants.TEZ_AM_LOCAL_RESOURCES_PB_FILE_NAME)); + return DAGProtos.PlanLocalResourcesProto.parseDelimitedFrom(sessionResourcesStream); + } finally { + if (sessionResourcesStream != null) { + sessionResourcesStream.close(); + } + } } @VisibleForTesting - public static void initAmRegistry(ApplicationId appId, String amUUID, AMRegistry amRegistry, - DAGClientServer dagClientServer) { - if (amRegistry != null) { - dagClientServer.registerServiceListener((service) -> { - if (service.isInState(STATE.STARTED)) { - AMRecord amRecord = AMRegistryUtils.recordForDAGClientServer(appId, amUUID, dagClientServer); - try { - amRegistry.add(amRecord); - } catch (Exception e) { - throw new RuntimeException(e); - } + public static void initAmRegistry(ApplicationId appId, AMRegistry amRegistry, DAGClientServer dagClientServer) throws Exception { + dagClientServer.registerServiceListener((service) -> { + if (service.isInState(STATE.STARTED)) { + AMRecord amRecord = amRegistry.createAmRecord( + appId, dagClientServer.getBindAddress().getHostName(), dagClientServer.getBindAddress().getPort() + ); + try { + amRegistry.add(amRecord); + LOG.info("Added AMRecord: {} to registry..", amRecord); + } catch (Exception e) { + throw new RuntimeException(e); } - }); - } + } + }); + } + + protected void initClientRpcServer() { + clientRpcServer = new DAGClientServer(clientHandler, appAttemptID, recoveryFS); + addIfService(clientRpcServer, true); } @VisibleForTesting @@ -869,7 +889,7 @@ protected synchronized void handle(DAGAppMasterEvent event) { } else { LOG.info("Session shutting down now."); this.taskSchedulerManager.setShouldUnregisterFlag(); - if (this.historyEventHandler.hasRecoveryFailed()) { + if (recoveryEnabled && this.historyEventHandler.hasRecoveryFailed()) { state = DAGAppMasterState.FAILED; } else { state = DAGAppMasterState.SUCCEEDED; @@ -2031,16 +2051,18 @@ public void serviceStart() throws Exception { this.lastDAGCompletionTime = clock.getTime(); - DAGRecoveryData recoveredDAGData; - try { - recoveredDAGData = recoverDAG(); - } catch (IOException e) { - LOG.error("Error occurred when trying to recover data from previous attempt." - + " Shutting down AM", e); - this.state = DAGAppMasterState.ERROR; - this.taskSchedulerManager.setShouldUnregisterFlag(); - shutdownHandler.shutdown(); - return; + DAGRecoveryData recoveredDAGData = null; + if(recoveryEnabled) { + try { + recoveredDAGData = recoverDAG(); + } catch (IOException e) { + LOG.error("Error occurred when trying to recover data from previous attempt." + + " Shutting down AM", e); + this.state = DAGAppMasterState.ERROR; + this.taskSchedulerManager.setShouldUnregisterFlag(); + shutdownHandler.shutdown(); + return; + } } DAGPlan dagPlan = null; @@ -2061,7 +2083,7 @@ public void serviceStart() throws Exception { this.state = DAGAppMasterState.IDLE; } - if (recoveredDAGData != null) { + if (recoveryEnabled && recoveredDAGData != null) { if (recoveredDAGData.cumulativeAdditionalResources != null) { recoveredDAGData.additionalUrlsForClasspath = processAdditionalResources( recoveredDAGData.recoveredDagID, @@ -2385,16 +2407,14 @@ public static void main(String[] args) { TezClassLoader.setupTezClassLoader(); Thread.setDefaultUncaughtExceptionHandler(new YarnUncaughtExceptionHandler()); final String pid = System.getenv().get("JVM_PID"); - String containerIdStr = - System.getenv(Environment.CONTAINER_ID.name()); - String nodeHostString = System.getenv(Environment.NM_HOST.name()); - String nodePortString = System.getenv(Environment.NM_PORT.name()); - String nodeHttpPortString = - System.getenv(Environment.NM_HTTP_PORT.name()); - String appSubmitTimeStr = - System.getenv(ApplicationConstants.APP_SUBMIT_TIME_ENV); + + String containerIdStr = System.getenv(ApplicationConstants.Environment.CONTAINER_ID.name()); + + String nodeHostString = System.getenv(ApplicationConstants.Environment.NM_HOST.name()); + String nodePortString = System.getenv(ApplicationConstants.Environment.NM_PORT.name()); + String nodeHttpPortString = System.getenv(ApplicationConstants.Environment.NM_HTTP_PORT.name()); + String appSubmitTimeStr = System.getenv(ApplicationConstants.APP_SUBMIT_TIME_ENV); String clientVersion = System.getenv(TezConstants.TEZ_CLIENT_VERSION_ENV); - String amUUID = System.getenv(TezConstants.TEZ_AM_UUID); if (clientVersion == null) { clientVersion = VersionInfo.UNKNOWN; } @@ -2402,16 +2422,25 @@ public static void main(String[] args) { Objects.requireNonNull(appSubmitTimeStr, ApplicationConstants.APP_SUBMIT_TIME_ENV + " is null"); - ContainerId containerId = ConverterUtils.toContainerId(containerIdStr); - ApplicationAttemptId applicationAttemptId = - containerId.getApplicationAttemptId(); + Configuration conf = new Configuration(); + + DAGProtos.ConfigurationProto confProto = amExts.flatMap(amExt -> amExt.loadConfigurationProto()).orElse(null); + if (confProto == null) { + confProto = TezUtilsInternal + .readUserSpecifiedTezConfiguration(System.getenv(ApplicationConstants.Environment.PWD.name())); + } + TezUtilsInternal.addUserSpecifiedTezConfiguration(conf, confProto.getConfKeyValuesList()); + + ContainerId containerId = amExts.flatMap(amExt -> amExt.allocateContainerId(conf)).orElse(null); + if (containerId == null) { + containerId = ConverterUtils.toContainerId(containerIdStr); + } + ApplicationAttemptId applicationAttemptId = containerId.getApplicationAttemptId(); org.apache.hadoop.ipc.CallerContext.setCurrent(new org.apache.hadoop.ipc.CallerContext .Builder("tez_appmaster_" + containerId.getApplicationAttemptId() ).build()); long appSubmitTime = Long.parseLong(appSubmitTimeStr); - - String jobUserName = System - .getenv(ApplicationConstants.Environment.USER.name()); + String jobUserName = System.getenv(ApplicationConstants.Environment.USER.name()); // Command line options Option option = Option.builder() @@ -2431,15 +2460,9 @@ public static void main(String[] args) { + ", jvmPid=" + pid + ", userFromEnv=" + jobUserName + ", cliSessionOption=" + sessionModeCliOption - + ", pwd=" + System.getenv(Environment.PWD.name()) - + ", localDirs=" + System.getenv(Environment.LOCAL_DIRS.name()) - + ", logDirs=" + System.getenv(Environment.LOG_DIRS.name())); - - Configuration conf = new Configuration(); - - ConfigurationProto confProto = - TezUtilsInternal.readUserSpecifiedTezConfiguration(System.getenv(Environment.PWD.name())); - TezUtilsInternal.addUserSpecifiedTezConfiguration(conf, confProto.getConfKeyValuesList()); + + ", pwd=" + System.getenv(ApplicationConstants.Environment.PWD.name()) + + ", localDirs=" + System.getenv(ApplicationConstants.Environment.LOCAL_DIRS.name()) + + ", logDirs=" + System.getenv(ApplicationConstants.Environment.LOG_DIRS.name())); AMPluginDescriptorProto amPluginDescriptorProto = null; if (confProto.hasAmPluginDescriptor()) { @@ -2452,16 +2475,13 @@ public static void main(String[] args) { TezUtilsInternal.setSecurityUtilConfigration(LOG, conf); DAGAppMaster appMaster = - new DAGAppMaster(applicationAttemptId, containerId, nodeHostString, - Integer.parseInt(nodePortString), - Integer.parseInt(nodeHttpPortString), new SystemClock(), appSubmitTime, - sessionModeCliOption, - System.getenv(Environment.PWD.name()), - TezCommonUtils.getTrimmedStrings(System.getenv(Environment.LOCAL_DIRS.name())), - TezCommonUtils.getTrimmedStrings(System.getenv(Environment.LOG_DIRS.name())), - clientVersion, credentials, jobUserName, amPluginDescriptorProto, amUUID); - ShutdownHookManager.get().addShutdownHook( - new DAGAppMasterShutdownHook(appMaster), SHUTDOWN_HOOK_PRIORITY); + new DAGAppMaster(applicationAttemptId, containerId, nodeHostString, Integer.parseInt(nodePortString), + Integer.parseInt(nodeHttpPortString), new SystemClock(), appSubmitTime, sessionModeCliOption, + System.getenv(ApplicationConstants.Environment.PWD.name()), + TezCommonUtils.getTrimmedStrings(System.getenv(ApplicationConstants.Environment.LOCAL_DIRS.name())), + TezCommonUtils.getTrimmedStrings(System.getenv(ApplicationConstants.Environment.LOG_DIRS.name())), + clientVersion, credentials, jobUserName, amPluginDescriptorProto); + ShutdownHookManager.get().addShutdownHook(new DAGAppMasterShutdownHook(appMaster), SHUTDOWN_HOOK_PRIORITY); // log the system properties if (LOG.isInfoEnabled()) { @@ -2518,7 +2538,6 @@ public void run() { } else if (appMaster.state == DAGAppMasterState.RUNNING) { appMaster.state = DAGAppMasterState.ERROR; } - appMaster.stop(); } @@ -2724,4 +2743,20 @@ public void vertexComplete(TezVertexID completedVertexID, Set nodesList) public void taskAttemptFailed(TezTaskAttemptID attemptID, NodeId nodeId) { getContainerLauncherManager().taskAttemptFailed(attemptID, jobTokenSecretManager, nodeId); } + + private static DAGProtos.ConfigurationProto getConfigurationProtoFromText() throws IOException { + Configuration configuration = new Configuration(); + String baseDir = System.getenv("TEZ_CONF_DIR"); + BufferedReader br = new BufferedReader( + new InputStreamReader(new FileInputStream(baseDir + '/' + "tez-site.xml"))); + StringBuilder builder = new StringBuilder(); + String line = br.readLine(); + while (line != null) { + builder.append(line); + line = br.readLine(); + } + byte[] bytes = builder.toString().getBytes(); + configuration.addResource(new ByteArrayInputStream(bytes)); + return TezClientUtils.createFinalConfProtoForApp(configuration, null); + } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/LocalDAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/LocalDAGAppMaster.java index 71eafd8965..e0c8443577 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/LocalDAGAppMaster.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/LocalDAGAppMaster.java @@ -37,7 +37,7 @@ public LocalDAGAppMaster(ApplicationAttemptId applicationAttemptId, ContainerId Credentials credentials, String jobUserName, AMPluginDescriptorProto pluginDescriptorProto) { super(applicationAttemptId, containerId, nmHost, nmPort, nmHttpPort, clock, appSubmitTime, isSession, workingDirectory, localDirs, logDirs, clientVersion, credentials, jobUserName, - pluginDescriptorProto, null); + pluginDescriptorProto); } @Override diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java b/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java index 04812dff17..7e2452525d 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java @@ -46,6 +46,7 @@ import org.apache.tez.dag.app.dag.impl.DAGImpl; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; +import org.apache.tez.dag.history.events.*; import org.apache.tez.dag.history.events.AMLaunchedEvent; import org.apache.tez.dag.history.events.AMStartedEvent; import org.apache.tez.dag.history.events.ContainerLaunchedEvent; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java index b2e750bb21..7794a07047 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java @@ -159,7 +159,8 @@ protected void startRpcServer() { conf.getInt(TezConfiguration.TEZ_AM_TASK_LISTENER_THREAD_COUNT, TezConfiguration.TEZ_AM_TASK_LISTENER_THREAD_COUNT_DEFAULT)) .setPortRangeConfig(TezConfiguration.TEZ_AM_TASK_AM_PORT_RANGE) - .setSecretManager(jobTokenSecretManager).build(); + .setSecretManager(jobTokenSecretManager) + .build(); // Enable service authorization? if (conf.getBoolean( diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java index e86624db78..5f08e33afe 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java @@ -92,8 +92,9 @@ import org.apache.tez.dag.api.records.DAGProtos.PlanVertexGroupInfo; import org.apache.tez.dag.api.records.DAGProtos.VertexPlan; import org.apache.tez.dag.app.AppContext; +import org.apache.tez.dag.app.DAGAppMaster; +import org.apache.tez.dag.app.RecoveryParser; import org.apache.tez.dag.app.RecoveryParser.DAGRecoveryData; -import org.apache.tez.dag.app.RecoveryParser.VertexRecoveryData; import org.apache.tez.dag.app.TaskCommunicatorManagerInterface; import org.apache.tez.dag.app.TaskHeartbeatHandler; import org.apache.tez.dag.app.dag.DAG; @@ -1290,7 +1291,7 @@ void logJobHistoryInitedEvent() { clock.getTime(), this.userName, this.dagName, this.getVertexNameIDMapping()); this.appContext.getHistoryHandler().handle( new DAGHistoryEvent(dagId, initEvt)); - } + } } void logJobHistoryStartedEvent() { @@ -1300,7 +1301,7 @@ void logJobHistoryStartedEvent() { this.startTime, this.userName, this.dagName); this.appContext.getHistoryHandler().handle( new DAGHistoryEvent(dagId, startEvt)); - } + } } void logJobHistoryFinishedEvent(TezCounters counters) throws IOException { @@ -1319,7 +1320,7 @@ void logJobHistoryFinishedEvent(TezCounters counters) throws IOException { this.jobPlan); this.appContext.getHistoryHandler().handleCriticalEvent( new DAGHistoryEvent(dagId, finishEvt)); - } + } } void logJobHistoryUnsuccesfulEvent(DAGState state, TezCounters counters) throws IOException { @@ -1339,7 +1340,7 @@ void logJobHistoryUnsuccesfulEvent(DAGState state, TezCounters counters) throws this.appContext.getApplicationAttemptId(), this.jobPlan); this.appContext.getHistoryHandler().handleCriticalEvent( new DAGHistoryEvent(dagId, finishEvt)); - } + } } // triggered by vertex_complete @@ -1613,7 +1614,11 @@ DAGState initializeDAG() { } // check task resources, only check it in non-local mode - if (!appContext.isLocal()) { + boolean checkTaskResources = true; + if(DAGAppMaster.amExts.isPresent()) { + checkTaskResources = DAGAppMaster.amExts.get().checkTaskResources(vertexMap, appContext); + } + if (!appContext.isLocal() && checkTaskResources) { for (Vertex v : vertexMap.values()) { // TODO TEZ-2003 (post) TEZ-2624 Ideally, this should be per source. if (v.getTaskResource().compareTo(appContext.getClusterInfo().getMaxContainerCapability()) > 0) { @@ -1770,21 +1775,21 @@ private static void parseVertexEdges(DAGImpl dag, Map edgePlan new HashMap(); for(String inEdgeId : vertexPlan.getInEdgeIdList()){ - EdgePlan edgePlan = edgePlans.get(inEdgeId); - Vertex inVertex = dag.vertexMap.get(edgePlan.getInputVertexName()); - Edge edge = dag.edges.get(inEdgeId); - edge.setSourceVertex(inVertex); - edge.setDestinationVertex(vertex); - inVertices.put(inVertex, edge); + EdgePlan edgePlan = edgePlans.get(inEdgeId); + Vertex inVertex = dag.vertexMap.get(edgePlan.getInputVertexName()); + Edge edge = dag.edges.get(inEdgeId); + edge.setSourceVertex(inVertex); + edge.setDestinationVertex(vertex); + inVertices.put(inVertex, edge); } for(String outEdgeId : vertexPlan.getOutEdgeIdList()){ - EdgePlan edgePlan = edgePlans.get(outEdgeId); - Vertex outVertex = dag.vertexMap.get(edgePlan.getOutputVertexName()); - Edge edge = dag.edges.get(outEdgeId); - edge.setSourceVertex(vertex); - edge.setDestinationVertex(outVertex); - outVertices.put(outVertex, edge); + EdgePlan edgePlan = edgePlans.get(outEdgeId); + Vertex outVertex = dag.vertexMap.get(edgePlan.getOutputVertexName()); + Edge edge = dag.edges.get(outEdgeId); + edge.setSourceVertex(vertex); + edge.setDestinationVertex(outVertex); + outVertices.put(outVertex, edge); } vertex.setInputVertices(inVertices); @@ -1803,7 +1808,7 @@ private static void parseVertexEdges(DAGImpl dag, Map edgePlan *
    *
  • * 1. For the completed dag, recover the dag to the desired state and also its vertices, - * but not task & task attempt. This recovery is sync call (after this Transition, + * but not task & task attempt. This recovery is sync call (after this Transition, * DAG & vertices are all recovered to the desired state) *
  • *
  • @@ -1861,7 +1866,6 @@ public DAGState transition(DAGImpl dag, DAGEvent dagEvent) { return DAGState.NEW; } } - private static class InitTransition implements MultipleArcTransition { @@ -1880,7 +1884,7 @@ public DAGState transition(DAGImpl dag, DAGEvent event) { if (dag.recoveryData != null && dag.recoveryData.getDAGInitializedEvent() != null) { dag.initTime = dag.recoveryData.getDAGInitializedEvent().getInitTime(); } else { - dag.initTime = dag.clock.getTime(); + dag.initTime = dag.clock.getTime(); } dag.startDAGCpuTime = dag.appContext.getCumulativeCPUTime(); dag.startDAGGCTime = dag.appContext.getCumulativeGCTime(); @@ -1912,7 +1916,7 @@ public void transition(DAGImpl dag, DAGEvent event) { if (dag.recoveryData != null && dag.recoveryData.getDAGStartedEvent() != null) { dag.startTime = dag.recoveryData.getDAGStartedEvent().getStartTime(); } else { - dag.startTime = dag.clock.getTime(); + dag.startTime = dag.clock.getTime(); } DAGEventStartDag startEvent = (DAGEventStartDag) event; List additionalUrlsForClasspath = startEvent.getAdditionalUrlsForClasspath(); @@ -2174,7 +2178,7 @@ private boolean vertexSucceeded(Vertex vertex) { LOG.info("VertexGroup was already committed as per recovery" + " data, groupName=" + groupInfo.groupName); for (String vertexName : groupInfo.groupMembers) { - VertexRecoveryData vertexRecoveryData = + RecoveryParser.VertexRecoveryData vertexRecoveryData = recoveryData.getVertexRecoveryData(getVertex(vertexName).getVertexId()); Preconditions.checkArgument(vertexRecoveryData != null,"Vertex Group has been committed" + ", but no VertexRecoveryData found for its vertex " + vertexName); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java index 3a28db07cf..8d69d86a4d 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java @@ -132,7 +132,7 @@ public class TaskAttemptImpl implements TaskAttempt, private static final Logger LOG = LoggerFactory.getLogger(TaskAttemptImpl.class); private static final String LINE_SEPARATOR = System .getProperty("line.separator"); - + public static class DataEventDependencyInfo { long timestamp; TezTaskAttemptID taId; @@ -576,7 +576,6 @@ public TaskAttemptImpl(TezTaskAttemptID attemptId, EventHandler eventHandler, this.hungIntervalMax = conf.getLong( TezConfiguration.TEZ_TASK_PROGRESS_STUCK_INTERVAL_MS, TezConfiguration.TEZ_TASK_PROGRESS_STUCK_INTERVAL_MS_DEFAULT); - this.recoveryData = appContext.getDAGRecoveryData() == null ? null : appContext.getDAGRecoveryData().getTaskAttemptRecoveryData(attemptId); } @@ -1267,7 +1266,7 @@ public TaskAttemptStateInternal transition(TaskAttemptImpl ta, TaskAttemptEvent return TaskAttemptStateInternal.NEW; } } - // No matter whether TaskAttemptStartedEvent is seen, send corresponding event to move + // No matter whether TaskAttemptStartedEvent is seen, send corresponding event to move // TA to the state of TaskAttemptFinishedEvent TaskAttemptFinishedEvent taFinishedEvent = ta.recoveryData.getTaskAttemptFinishedEvent(); @@ -1298,7 +1297,6 @@ public TaskAttemptStateInternal transition(TaskAttemptImpl ta, TaskAttemptEvent } return TaskAttemptStateInternal.NEW; } - TaskAttemptEventSchedule scheduleEvent = (TaskAttemptEventSchedule) event; ta.scheduledTime = ta.clock.getTime(); @@ -1402,9 +1400,9 @@ public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) { if (ta.recoveryData == null || ta.recoveryData.getTaskAttemptFinishedEvent() == null) { - ta.setFinishTime(); - ta.logJobHistoryAttemptUnsuccesfulCompletion(helper - .getTaskAttemptState(), helper.getFailureType(event)); + ta.setFinishTime(); + ta.logJobHistoryAttemptUnsuccesfulCompletion(helper + .getTaskAttemptState(), helper.getFailureType(event)); } else { ta.finishTime = ta.recoveryData.getTaskAttemptFinishedEvent().getFinishTime(); ta.isRecoveredDuration = true; @@ -1637,6 +1635,7 @@ public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) { // If TaskAttempt is recovered to SUCCEEDED, send events generated by this TaskAttempt to vertex // for its downstream consumers. For normal dag execution, the events are sent by TaskAttemptListener // for performance consideration. + if (ta.recoveryData != null && ta.recoveryData.isTaskAttemptSucceeded()) { TaskAttemptFinishedEvent taFinishedEvent = ta.recoveryData .getTaskAttemptFinishedEvent(); @@ -1649,9 +1648,9 @@ public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) { ta.finishTime = taFinishedEvent.getFinishTime(); ta.isRecoveredDuration = true; } else { - ta.setFinishTime(); - // Send out history event. - ta.logJobHistoryAttemptFinishedEvent(TaskAttemptStateInternal.SUCCEEDED); + ta.setFinishTime(); + // Send out history event. + ta.logJobHistoryAttemptFinishedEvent(TaskAttemptStateInternal.SUCCEEDED); } ta.sendEvent(createDAGCounterUpdateEventTAFinished(ta, diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java index ffb41e364a..6a78ea86eb 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java @@ -55,7 +55,7 @@ import org.apache.tez.dag.api.oldrecords.TaskState; import org.apache.tez.dag.app.AppContext; import org.apache.tez.dag.app.ContainerContext; -import org.apache.tez.dag.app.RecoveryParser.TaskRecoveryData; +import org.apache.tez.dag.app.RecoveryParser; import org.apache.tez.dag.app.TaskCommunicatorManagerInterface; import org.apache.tez.dag.app.TaskHeartbeatHandler; import org.apache.tez.dag.app.dag.StateChangeNotifier; @@ -143,8 +143,7 @@ public class TaskImpl implements Task, EventHandler { long scheduledTime; final StateChangeNotifier stateChangeNotifier; - private final TaskRecoveryData recoveryData; - + private final RecoveryParser.TaskRecoveryData recoveryData; private final List tezEventsForTaskAttempts = new ArrayList(); static final ArrayList EMPTY_TASK_ATTEMPT_TEZ_EVENTS = new ArrayList(0); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezLocalCacheManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezLocalCacheManager.java index 2d044a5689..a0268f66e1 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezLocalCacheManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezLocalCacheManager.java @@ -80,9 +80,8 @@ public void localize() throws IOException { try { // construct new threads with helpful names - ThreadFactory threadFactory = new ThreadFactoryBuilder() - .setNameFormat("TezLocalCacheManager Downloader #%d") - .build(); + ThreadFactory threadFactory = + new ThreadFactoryBuilder().setNameFormat("TezLocalCacheManager Downloader #%d").build(); threadPool = Executors.newCachedThreadPool(threadFactory); // start all fetches @@ -108,6 +107,7 @@ public void localize() throws IOException { Path dest = new Path(downloadDir.toAbsolutePath().toString()); FSDownload downloader = new FSDownload(fileContext, ugi, conf, dest, resource); Future downloadedPath = threadPool.submit(downloader); + resourceInfo.put(resource, new ResourceInfo(downloadedPath, linkPath)); } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java index c9c6178732..19af76aa6e 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java @@ -644,7 +644,7 @@ protected void instantiateSchedulers(String host, int port, String trackingUrl, // Use the provided appId instead of constructing one for containers. customAppIdIdentifier = appContext.getApplicationID().getClusterTimestamp(); } else { - customAppIdIdentifier = SCHEDULER_APP_ID_BASE + (j++ * SCHEDULER_APP_ID_INCREMENT); + customAppIdIdentifier = SCHEDULER_APP_ID_BASE + (j++ * SCHEDULER_APP_ID_INCREMENT); } taskSchedulers[i] = new TaskSchedulerWrapper(createTaskScheduler(host, port, trackingUrl, appContext, taskSchedulerDescriptors[i], customAppIdIdentifier, i)); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/utils/AMRegistryUtils.java b/tez-dag/src/main/java/org/apache/tez/dag/utils/AMRegistryUtils.java deleted file mode 100644 index 13cc27cbac..0000000000 --- a/tez-dag/src/main/java/org/apache/tez/dag/utils/AMRegistryUtils.java +++ /dev/null @@ -1,45 +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.tez.dag.utils; - -import java.net.InetSocketAddress; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.tez.client.registry.AMRecord; -import org.apache.tez.common.ReflectionUtils; -import org.apache.tez.dag.api.TezConfiguration; -import org.apache.tez.dag.api.client.DAGClientServer; -import org.apache.tez.dag.api.client.registry.AMRegistry; - -public final class AMRegistryUtils { - - private AMRegistryUtils() {} - - public static AMRecord recordForDAGClientServer(ApplicationId appId, String opaqueId, - DAGClientServer dagClientServer) { - InetSocketAddress address = dagClientServer.getBindAddress(); - return new AMRecord(appId, address.getHostName(), address.getPort(), opaqueId); - } - - public static AMRegistry createAMRegistry(Configuration conf) throws Exception { - String tezAMRegistryClass = conf.get(TezConfiguration.TEZ_AM_REGISTRY_CLASS); - return tezAMRegistryClass == null ? null : ReflectionUtils.createClazzInstance(tezAMRegistryClass); - } -} diff --git a/tez-dag/src/main/java/org/apache/tez/frameworkplugins/AmExtensions.java b/tez-dag/src/main/java/org/apache/tez/frameworkplugins/AmExtensions.java new file mode 100644 index 0000000000..5782944281 --- /dev/null +++ b/tez-dag/src/main/java/org/apache/tez/frameworkplugins/AmExtensions.java @@ -0,0 +1,63 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.tez.frameworkplugins; + +import java.util.Map; +import java.util.Optional; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.Credentials; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.tez.common.security.JobTokenIdentifier; +import org.apache.tez.common.security.JobTokenSecretManager; +import org.apache.tez.dag.api.records.DAGProtos; +import org.apache.tez.dag.app.AppContext; +import org.apache.tez.dag.app.dag.Vertex; + +/* + Plugin points to provide alternate AM behavior that + is either too small or too scattered to be usefully encapsulated as its own service + */ +public interface AmExtensions { + + //Override default Configuration loading at DAGAppMaster.main + default Optional loadConfigurationProto() { return Optional.empty(); } + + //Override default behavior to give ContainerId to AM + default Optional allocateContainerId(Configuration conf) { return Optional.empty(); } + + //Whether this framework requires addition of the default Yarn ServicePlugins + default boolean isUsingYarnServicePlugin() { + return true; + } + + //Whether to check task resources against ClusterInfo + default boolean checkTaskResources(Map vertices, AppContext appContext) { return true; } + + default Optional> getSessionToken( + ApplicationAttemptId appAttemptID, + JobTokenSecretManager jobTokenSecretManager, + Credentials amCredentials + ) { return Optional.empty(); } + + default Optional getAdditionalSessionResources(String dir) { + return Optional.empty(); + } +} diff --git a/tez-dag/src/main/java/org/apache/tez/frameworkplugins/ServerFrameworkService.java b/tez-dag/src/main/java/org/apache/tez/frameworkplugins/ServerFrameworkService.java new file mode 100644 index 0000000000..f776d4eb9d --- /dev/null +++ b/tez-dag/src/main/java/org/apache/tez/frameworkplugins/ServerFrameworkService.java @@ -0,0 +1,32 @@ +/** + * 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.tez.frameworkplugins; + +import java.util.Optional; + +import org.apache.hadoop.conf.Configuration; +import org.apache.tez.client.registry.AMRegistry; + +/* + FrameworkService that runs code within the AM process launched from DAGAppMaster.main(..) + Bundles together an AMRegistry and AmExtensions impl. that are compatible + */ +public interface ServerFrameworkService extends FrameworkService { + Optional createOrGetAMRegistry(Configuration conf); + Optional createOrGetDAGAppMasterExtensions(); +} diff --git a/tez-dag/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZkStandaloneAmExtensions.java b/tez-dag/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZkStandaloneAmExtensions.java new file mode 100644 index 0000000000..6a86914bb2 --- /dev/null +++ b/tez-dag/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZkStandaloneAmExtensions.java @@ -0,0 +1,104 @@ +/** + * 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.tez.frameworkplugins.zookeeper; + +import java.io.IOException; +import java.util.Map; +import java.util.Optional; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.security.Credentials; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.tez.client.registry.AMRegistry; +import org.apache.tez.common.TezUtilsInternal; +import org.apache.tez.common.security.JobTokenIdentifier; +import org.apache.tez.common.security.JobTokenSecretManager; +import org.apache.tez.common.security.TokenCache; +import org.apache.tez.dag.api.records.DAGProtos; +import org.apache.tez.dag.app.AppContext; +import org.apache.tez.dag.app.dag.Vertex; +import org.apache.tez.frameworkplugins.AmExtensions; +import org.apache.tez.frameworkplugins.ServerFrameworkService; + +public class ZkStandaloneAmExtensions implements AmExtensions { + + ServerFrameworkService myFrameworkService; + + public ZkStandaloneAmExtensions(ServerFrameworkService myFrameworkService) { + this.myFrameworkService = myFrameworkService; + } + + @Override public Optional allocateContainerId(Configuration conf) { + try { + Optional amRegistry = myFrameworkService.createOrGetAMRegistry(conf); + if(amRegistry.isPresent()) { + ApplicationId appId = amRegistry.get().generateNewId().get(); + // attempId is set to 1 only then APP_LAUNCHED event gets triggered + ApplicationAttemptId applicationAttemptId = ApplicationAttemptId.newInstance(appId, 1); + return Optional.of(ContainerId.newContainerId(applicationAttemptId, 0)); + } else { + throw new RuntimeException("AMRegistry is required for ZkStandaloneAmExtensions"); + } + + } catch(Exception e) { + throw new RuntimeException(e); + } + } + + @Override public boolean checkTaskResources(Map vertices, AppContext appContext) { + return false; + } + + @Override + public boolean isUsingYarnServicePlugin() { + return false; + } + + @Override public Optional loadConfigurationProto() { + try { + return Optional.of(TezUtilsInternal.loadConfProtoFromText()); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public Optional> getSessionToken( + ApplicationAttemptId appAttemptID, + JobTokenSecretManager jobTokenSecretManager, + Credentials amCredentials + ) { + JobTokenIdentifier identifier = new JobTokenIdentifier(new Text( + appAttemptID.getApplicationId().toString())); + Token newSessionToken = new Token(identifier, + jobTokenSecretManager); + newSessionToken.setService(identifier.getJobId()); + TokenCache.setSessionToken(newSessionToken, amCredentials); + return Optional.of(newSessionToken); + } + + @Override + public Optional getAdditionalSessionResources(String dir) { + return Optional.of(DAGProtos.PlanLocalResourcesProto.getDefaultInstance()); + } + +} diff --git a/tez-dag/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZookeeperStandaloneServerFrameworkService.java b/tez-dag/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZookeeperStandaloneServerFrameworkService.java new file mode 100644 index 0000000000..877ed64cf7 --- /dev/null +++ b/tez-dag/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZookeeperStandaloneServerFrameworkService.java @@ -0,0 +1,39 @@ +package org.apache.tez.frameworkplugins.zookeeper; + +import java.util.Optional; + +import org.apache.hadoop.conf.Configuration; +import org.apache.tez.client.registry.AMRegistry; +import org.apache.tez.dag.api.TezConstants; +import org.apache.tez.dag.api.client.registry.zookeeper.ZkAMRegistry; +import org.apache.tez.frameworkplugins.AmExtensions; +import org.apache.tez.frameworkplugins.ServerFrameworkService; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ZookeeperStandaloneServerFrameworkService implements ServerFrameworkService { + private static final Logger LOG = LoggerFactory.getLogger(ZookeeperStandaloneServerFrameworkService.class); + private ZkAMRegistry amRegistry; + + @Override + public synchronized Optional createOrGetAMRegistry(Configuration conf) { + if (amRegistry == null) { + try { + final String externalID = System.getenv(TezConstants.TEZ_AM_EXTERNAL_ID); + amRegistry = new ZkAMRegistry(externalID); + amRegistry.init(conf); + amRegistry.start(); + LOG.info("Created Zookeeper based AM Registry with externalID: {}", externalID); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + return Optional.of(amRegistry); + } + + @Override + public Optional createOrGetDAGAppMasterExtensions() { + return Optional.of(new ZkStandaloneAmExtensions(this)); + } +} diff --git a/tez-dag/src/test/java/org/apache/tez/dag/api/client/registry/TestAMRegistry.java b/tez-dag/src/test/java/org/apache/tez/dag/api/client/registry/TestAMRegistry.java deleted file mode 100644 index dc8cc4acf7..0000000000 --- a/tez-dag/src/test/java/org/apache/tez/dag/api/client/registry/TestAMRegistry.java +++ /dev/null @@ -1,100 +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.tez.dag.api.client.registry; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -import java.net.InetSocketAddress; -import java.util.UUID; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; -import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.tez.client.registry.AMRecord; -import org.apache.tez.dag.api.TezConfiguration; -import org.apache.tez.dag.api.client.DAGClientHandler; -import org.apache.tez.dag.api.client.DAGClientServer; -import org.apache.tez.dag.app.DAGAppMaster; -import org.apache.tez.dag.utils.AMRegistryUtils; - -import org.junit.Test; - -public class TestAMRegistry { - - @Test(timeout = 5000) - public void testAMRegistryFactory() throws Exception { - Configuration conf = new Configuration(); - AMRegistry amRegistry = AMRegistryUtils.createAMRegistry(conf); - assertNull(amRegistry); - String className = SkeletonAMRegistry.class.getName(); - conf.set(TezConfiguration.TEZ_AM_REGISTRY_CLASS, className); - amRegistry = AMRegistryUtils.createAMRegistry(conf); - assertNotNull(amRegistry); - assertEquals(className, amRegistry.getClass().getName()); - } - - @Test(timeout = 5000) - public void testRecordForDagServer() { - DAGClientServer dagClientServer = mock(DAGClientServer.class); - when(dagClientServer.getBindAddress()).thenReturn(new InetSocketAddress("testhost", 1000)); - ApplicationId appId = ApplicationId.newInstance(0, 1); - String id = UUID.randomUUID().toString(); - AMRecord record = AMRegistryUtils.recordForDAGClientServer(appId, id, dagClientServer); - assertEquals(appId, record.getApplicationId()); - assertEquals("testhost", record.getHost()); - assertEquals(1000, record.getPort()); - assertEquals(record.getId(), id); - } - - @Test(timeout = 20000) - public void testAMRegistryService() throws Exception { - DAGClientHandler dagClientHandler = mock(DAGClientHandler.class); - ApplicationAttemptId appAttemptId = mock(ApplicationAttemptId.class); - ApplicationId appId = ApplicationId.newInstance(0, 1); - String uuid = UUID.randomUUID().toString(); - when(appAttemptId.getApplicationId()).thenReturn(appId); - AMRegistry amRegistry = mock(AMRegistry.class); - FileSystem fs = mock(FileSystem.class); - DAGClientServer dagClientServer = new DAGClientServer(dagClientHandler, appAttemptId, fs); - try { - DAGAppMaster.initAmRegistry(appAttemptId.getApplicationId(), uuid, amRegistry, dagClientServer); - dagClientServer.init(new Configuration()); - dagClientServer.start(); - AMRecord record = AMRegistryUtils.recordForDAGClientServer(appId, uuid, dagClientServer); - verify(amRegistry, times(1)).add(record); - } finally { - dagClientServer.stop(); - } - } - - public static class SkeletonAMRegistry extends AMRegistry { - public SkeletonAMRegistry() { - super("SkeletonAMRegistry"); - } - @Override public void add(AMRecord server) throws Exception { } - @Override public void remove(AMRecord server) throws Exception { } - } -} diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java index 213d85b892..fbab519376 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java @@ -500,7 +500,7 @@ public MockDAGAppMaster(ApplicationAttemptId applicationAttemptId, ContainerId c Credentials credentials, String jobUserName, int handlerConcurrency, int numConcurrentContainers) { super(applicationAttemptId, containerId, nmHost, nmPort, nmHttpPort, clock, appSubmitTime, isSession, workingDirectory, localDirs, logDirs, new TezApiVersionInfo().getVersion(), - credentials, jobUserName, null, null); + credentials, jobUserName, null); shutdownHandler = new MockDAGAppMasterShutdownHandler(); this.launcherGoFlag = launcherGoFlag; this.initFailFlag = initFailFlag; diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java index afe2e8bc3b..0bc362db09 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java @@ -498,7 +498,7 @@ public void testBadProgress() throws Exception { TEST_DIR.toString(), new String[] {TEST_DIR.toString()}, new String[] {TEST_DIR.toString()}, new TezApiVersionInfo().getVersion(), amCreds, - "someuser", null, null)); + "someuser", null)); when(am.getState()).thenReturn(DAGAppMasterState.RUNNING); am.init(conf); am.start(); @@ -583,7 +583,7 @@ private void testDagCredentials(boolean doMerge) throws IOException { TEST_DIR.toString(), new String[] {TEST_DIR.toString()}, new String[] {TEST_DIR.toString()}, new TezApiVersionInfo().getVersion(), amCreds, - "someuser", null, null); + "someuser", null); am.init(conf); am.start(); @@ -704,7 +704,7 @@ public DAGAppMasterForTest(ApplicationAttemptId attemptId, boolean isSession) { super(attemptId, ContainerId.newContainerId(attemptId, 1), "hostname", 12345, 12346, new SystemClock(), 0, isSession, TEST_DIR.getAbsolutePath(), new String[] { TEST_DIR.getAbsolutePath() }, new String[] { TEST_DIR.getAbsolutePath() }, - new TezDagVersionInfo().getVersion(), createCredentials(), "jobname", null, null); + new TezDagVersionInfo().getVersion(), createCredentials(), "jobname", null); } public static Credentials createCredentials() { diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexManager.java index 97ba83652d..8778f2b4e9 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexManager.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexManager.java @@ -277,7 +277,6 @@ public void testSendCustomProcessorEvent() throws Exception { verify(mockHandler, times(1)).handle(requestCaptor.capture()); CustomProcessorEvent cpe = (CustomProcessorEvent)(requestCaptor.getValue().getEvents().get(0).getEvent()); - // should be able to get payload any times for (int i = 0; i < 2; i++) { ByteBuffer payloadBuffer = cpe.getPayload(); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java index 277291110c..0c808af0b6 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java @@ -597,7 +597,6 @@ static TaskSchedulerContext setupMockTaskSchedulerContext(String appHost, int ap throw new TezUncheckedException(e); } when(mockContext.getInitialUserPayload()).thenReturn(userPayload); - when(mockContext.isSession()).thenReturn(isSession); if (containerSignatureMatcher != null) { when(mockContext.getContainerSignatureMatcher()) .thenReturn(containerSignatureMatcher); diff --git a/tez-examples/src/main/java/org/apache/tez/examples/ExampleBase.java b/tez-examples/src/main/java/org/apache/tez/examples/ExampleBase.java new file mode 100644 index 0000000000..e9c6cb69e3 --- /dev/null +++ b/tez-examples/src/main/java/org/apache/tez/examples/ExampleBase.java @@ -0,0 +1,40 @@ +package org.apache.tez.examples; + +import java.io.IOException; + +import org.apache.tez.client.TezClient; +import org.apache.tez.dag.api.DAG; +import org.apache.tez.dag.api.TezConfiguration; +import org.apache.tez.dag.api.TezException; + +import org.slf4j.Logger; + +public abstract class ExampleBase { + /** + * Validate the arguments + * + * @param otherArgs arguments, if any + * @return Zero indicates success, non-zero indicates failure + */ + protected abstract int validateArgs(String[] otherArgs); + + /** + * Print usage instructions for this example + */ + protected abstract void printUsage(); + + /** + * Create and execute the actual DAG for the example + * + * @param args arguments for execution + * @param tezConf the tez configuration instance to be used while processing the DAG + * @param tezClient the tez client instance to use to run the DAG if any custom monitoring is + * required. Otherwise the utility method {@link #runDag(DAG, + * boolean, Logger)} should be used + * @return Zero indicates success, non-zero indicates failure + * @throws IOException + * @throws TezException + */ + protected abstract int runJob(String[] args, TezConfiguration tezConf, + TezClient tezClient) throws Exception; +} diff --git a/tez-examples/src/main/java/org/apache/tez/examples/StandaloneWordCount.java b/tez-examples/src/main/java/org/apache/tez/examples/StandaloneWordCount.java new file mode 100644 index 0000000000..47a8c75736 --- /dev/null +++ b/tez-examples/src/main/java/org/apache/tez/examples/StandaloneWordCount.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.tez.examples; + +import java.io.IOException; + +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.Text; +import org.apache.tez.client.TezClient; +import org.apache.tez.dag.api.*; +import org.apache.tez.mapreduce.SumProcessor; +import org.apache.tez.mapreduce.TokenProcessor; +import org.apache.tez.runtime.api.*; +import org.apache.tez.runtime.library.FakeInput; +import org.apache.tez.runtime.library.FakeInputInitializer; +import org.apache.tez.runtime.library.FakeOutput; +import org.apache.tez.runtime.library.FakeOutputCommitter; +import org.apache.tez.runtime.library.conf.OrderedPartitionedKVEdgeConfig; +import org.apache.tez.runtime.library.partitioner.HashPartitioner; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Simple example to perform WordCount using Tez API's. WordCount is the + * HelloWorld program of distributed data processing and counts the number + * of occurrences of a word in a distributed text data set. + */ +public class StandaloneWordCount extends ExampleBase { + + private static final Logger LOG = LoggerFactory.getLogger(StandaloneWordCount.class); + static String INPUT = "Input"; + static String SUMMATION = "Summation"; + static String OUTPUT = "Output"; + static String TOKENIZER = "Tokenizer"; + + /* + * Example code to write a processor in Tez. + * Processors typically apply the main application logic to the data. + * TokenProcessor tokenizes the input data. + * It uses an input that provide a Key-Value reader and writes + * output to a Key-Value writer. The processor inherits from SimpleProcessor + * since it does not need to handle any advanced constructs for Processors. + */ + + /* + * Example code to write a processor that commits final output to a data sink + * The SumProcessor aggregates the sum of individual word counts generated by + * the TokenProcessor. + * The SumProcessor is connected to a DataSink. In this case, its an Output that + * writes the data via an OutputFormat to a data sink (typically HDFS). Thats why + * it derives from SimpleMRProcessor that takes care of handling the necessary + * output commit operations that makes the final output available for consumers. + */ + + private DAG createDAG(TezConfiguration tezConf, String inputPath, String outputPath, + int numPartitions) throws IOException { + + InputDescriptor inputDescriptor = InputDescriptor.create(FakeInput.class.getName()); + InputInitializerDescriptor inputInitializerDescriptor = + InputInitializerDescriptor.create(FakeInputInitializer.class.getName()); + DataSourceDescriptor dataSource = + DataSourceDescriptor.create(inputDescriptor, inputInitializerDescriptor, null); + + OutputDescriptor outputDescriptor = OutputDescriptor.create(FakeOutput.class.getName()); + OutputCommitterDescriptor outputCommitterDescriptor = + OutputCommitterDescriptor.create(FakeOutputCommitter.class.getName()); + DataSinkDescriptor dataSink = + DataSinkDescriptor.create(outputDescriptor, outputCommitterDescriptor, null); + + Vertex tokenizerVertex = Vertex.create(TOKENIZER, ProcessorDescriptor.create( + TokenProcessor.class.getName())).addDataSource(INPUT, dataSource); + + OrderedPartitionedKVEdgeConfig edgeConf = OrderedPartitionedKVEdgeConfig + .newBuilder(Text.class.getName(), IntWritable.class.getName(), + HashPartitioner.class.getName()) + .setFromConfiguration(tezConf) + .build(); + + Vertex summationVertex = Vertex.create(SUMMATION, + ProcessorDescriptor.create(SumProcessor.class.getName()), numPartitions) + .addDataSink(OUTPUT, dataSink); + + DAG dag = DAG.create("WordCount"); + dag.addVertex(tokenizerVertex) + .addVertex(summationVertex) + .addEdge( + Edge.create(tokenizerVertex, summationVertex, edgeConf.createDefaultEdgeProperty())); + return dag; + } + + @Override + protected void printUsage() { + System.err.println("Usage: " + " wordcount in out"); + } + + @Override + protected int validateArgs(String[] otherArgs) { + if (otherArgs.length != 2 ) { + return 2; + } + return 0; + } + + @Override + protected int runJob(String[] args, TezConfiguration tezConf, + TezClient tezClient) throws Exception { + DAG dag = createDAG(tezConf, null, null,1); + LOG.info("Running WordCount"); + return StandaloneWordCountDriver.runDag(dag, tezClient.getAppMasterApplicationId() , LOG); + } + +} diff --git a/tez-examples/src/main/java/org/apache/tez/examples/StandaloneWordCountDriver.java b/tez-examples/src/main/java/org/apache/tez/examples/StandaloneWordCountDriver.java new file mode 100644 index 0000000000..e58760a7d2 --- /dev/null +++ b/tez-examples/src/main/java/org/apache/tez/examples/StandaloneWordCountDriver.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.tez.examples; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Set; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.security.Credentials; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.security.token.TokenIdentifier; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.tez.client.CallerContext; +import org.apache.tez.client.TezClient; +import org.apache.tez.client.registry.AMRecord; +import org.apache.tez.client.registry.zookeeper.ZkAMRegistryClient; +import org.apache.tez.dag.api.DAG; +import org.apache.tez.dag.api.TezConfiguration; +import org.apache.tez.dag.api.TezException; +import org.apache.tez.dag.api.client.DAGClient; +import org.apache.tez.dag.api.client.DAGStatus; +import org.apache.tez.dag.api.client.StatusGetOpts; + +import com.google.common.collect.Sets; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Private +public abstract class StandaloneWordCountDriver { + + private static final Logger LOG = LoggerFactory.getLogger(StandaloneWordCountDriver.class); + + private static TezClient tezClientInternal; + + public static void main(String[] args) throws Exception { + ExampleBase clazz = new StandaloneWordCount(); + _execute(clazz, args, null, null); + } + + private static int _execute(ExampleBase clazz, String[] otherArgs, TezConfiguration tezConf, TezClient tezClient) throws + Exception { + tezConf = new TezConfiguration(); + tezConf.set(TezConfiguration.TEZ_AM_ZOOKEEPER_QUORUM, "localhost:2181"); + ZkAMRegistryClient registryClientZk = ZkAMRegistryClient.getClient(tezConf); + registryClientZk.start(); + List sessions = registryClientZk.getAllRecords(); + Collections.shuffle(sessions); + AMRecord am = sessions.get(0); + tezConf.set(TezConfiguration.TEZ_FRAMEWORK_MODE, "STANDALONE_ZOOKEEPER"); + tezConf.setBoolean(TezConfiguration.TEZ_IGNORE_LIB_URIS, true); + tezClientInternal = createTezClient(am.getApplicationId().toString(), tezConf); + StandaloneWordCount standaloneWordCount = new StandaloneWordCount(); + return standaloneWordCount.runJob(otherArgs, tezConf, tezClientInternal); + } + + public static int runDag(DAG dag, ApplicationId appId, Logger logger) throws Exception { + //tezClientInternal.waitTillReady(); + + CallerContext callerContext = CallerContext.create("TezExamples", + "Tez Example DAG: " + dag.getName()); + + if (appId != null) { + callerContext.setCallerIdAndType(appId.toString(), "TezExampleApplication"); + } + dag.setCallerContext(callerContext); + + DAGClient dagClient = tezClientInternal.submitDAG(dag); + Set getOpts = Sets.newHashSet(); + getOpts.add(StatusGetOpts.GET_COUNTERS); + + DAGStatus dagStatus; + dagStatus = dagClient.waitForCompletionWithStatusUpdates(getOpts); + + if (dagStatus.getState() != DAGStatus.State.SUCCEEDED) { + logger.info("DAG diagnostics: " + dagStatus.getDiagnostics()); + return -1; + } + return 0; + } + + private static TezClient createTezClient(String appId, TezConfiguration tezConf) throws IOException, TezException { + Credentials credentials = new Credentials(); + Token token = new Token(); + credentials.addToken(new Text("root"), token); + TezClient tezClient = TezClient.create("TezExampleApplication", tezConf, true, null, credentials); + return tezClient.getClient(appId); + } + +} diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/SumProcessor.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/SumProcessor.java new file mode 100644 index 0000000000..2054f19233 --- /dev/null +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/SumProcessor.java @@ -0,0 +1,40 @@ +package org.apache.tez.mapreduce; + +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.Text; +import org.apache.tez.mapreduce.processor.SimpleMRProcessor; +import org.apache.tez.runtime.api.ProcessorContext; +import org.apache.tez.runtime.library.api.KeyValueWriter; +import org.apache.tez.runtime.library.api.KeyValuesReader; + +import com.google.common.base.Preconditions; + +public class SumProcessor extends SimpleMRProcessor { + public SumProcessor(ProcessorContext context) { + super(context); + } + + static String OUTPUT = "Output"; + static String TOKENIZER = "Tokenizer"; + + @Override + public void run() throws Exception { + Preconditions.checkArgument(getInputs().size() == 1); + Preconditions.checkArgument(getOutputs().size() == 1); + KeyValueWriter kvWriter = (KeyValueWriter) getOutputs().get(OUTPUT).getWriter(); + // The KeyValues reader provides all values for a given key. The aggregation of values per key + // is done by the LogicalInput. Since the key is the word and the values are its counts in + // the different TokenProcessors, summing all values per key provides the sum for that word. + KeyValuesReader kvReader = (KeyValuesReader) getInputs().get(TOKENIZER).getReader(); + while (kvReader.next()) { + Text word = (Text) kvReader.getCurrentKey(); + int sum = 0; + for (Object value : kvReader.getCurrentValues()) { + sum += ((IntWritable) value).get(); + } + kvWriter.write(word, new IntWritable(sum)); + } + // deriving from SimpleMRProcessor takes care of committing the output + // It automatically invokes the commit logic for the OutputFormat if necessary. + } +} diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/TokenProcessor.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/TokenProcessor.java new file mode 100644 index 0000000000..f35f24545e --- /dev/null +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/TokenProcessor.java @@ -0,0 +1,45 @@ +package org.apache.tez.mapreduce; + +import java.util.StringTokenizer; + +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.Text; +import org.apache.tez.runtime.api.ProcessorContext; +import org.apache.tez.runtime.library.api.KeyValueReader; +import org.apache.tez.runtime.library.api.KeyValueWriter; +import org.apache.tez.runtime.library.processor.SimpleProcessor; + +import com.google.common.base.Preconditions; + +public class TokenProcessor extends SimpleProcessor { + IntWritable one = new IntWritable(1); + Text word = new Text(); + static String INPUT = "Input"; + static String SUMMATION = "Summation"; + + public TokenProcessor(ProcessorContext context) { + super(context); + } + + @Override + public void run() throws Exception { + Preconditions.checkArgument(getInputs().size() == 1); + Preconditions.checkArgument(getOutputs().size() == 1); + // the recommended approach is to cast the reader/writer to a specific type instead + // of casting the input/output. This allows the actual input/output type to be replaced + // without affecting the semantic guarantees of the data type that are represented by + // the reader and writer. + // The inputs/outputs are referenced via the names assigned in the DAG. + KeyValueReader kvReader = (KeyValueReader) getInputs().get(INPUT).getReader(); + KeyValueWriter kvWriter = (KeyValueWriter) getOutputs().get(SUMMATION).getWriter(); + while (kvReader.next()) { + StringTokenizer itr = new StringTokenizer(kvReader.getCurrentValue().toString()); + while (itr.hasMoreTokens()) { + word.set(itr.nextToken()); + // Count 1 every time a word is observed. Word is the key a 1 is the value + kvWriter.write(word, one); + } + } + } + +} diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/FakeInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/FakeInput.java new file mode 100644 index 0000000000..3e2d644d72 --- /dev/null +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/FakeInput.java @@ -0,0 +1,80 @@ +package org.apache.tez.runtime.library; + +import java.io.IOException; +import java.util.List; + +import org.apache.tez.runtime.api.AbstractLogicalInput; +import org.apache.tez.runtime.api.Event; +import org.apache.tez.runtime.api.InputContext; +import org.apache.tez.runtime.api.Reader; +import org.apache.tez.runtime.library.api.KeyValueReader; + +public class FakeInput extends AbstractLogicalInput { + + private static final int numRecordPerSrc = 10; + + /** + * Constructor an instance of the LogicalInput. Classes extending this one to create a + * LogicalInput, must provide the same constructor so that Tez can create an instance of the + * class at runtime. + * + * @param inputContext the {@link InputContext} which provides + * the Input with context information within the running task. + * @param numPhysicalInputs the number of physical inputs that the logical input will + */ + public FakeInput(InputContext inputContext, int numPhysicalInputs) { + super(inputContext, numPhysicalInputs); + } + + @Override + public List initialize() throws Exception { + getContext().requestInitialMemory(0, null); + getContext().inputIsReady(); + return null; + } + + @Override + public void handleEvents(List inputEvents) throws Exception { + + } + + @Override + public List close() throws Exception { + return null; + } + + @Override + public void start() throws Exception { + + } + + @Override + public Reader getReader() throws Exception { + return new KeyValueReader() { + String[] keys = new String[numRecordPerSrc]; + + int i = -1; + + @Override + public boolean next() throws IOException { + if (i == -1) { + for (int j = 0; j < numRecordPerSrc; j++) { + keys[j] = ""+j; + } + } + i++; + return i < keys.length; + } + + @Override + public Object getCurrentKey() throws IOException { + return keys[i]; + } + + @Override + public Object getCurrentValue() throws IOException { + return keys[i]; + } + }; + } +} diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/FakeInputInitializer.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/FakeInputInitializer.java new file mode 100644 index 0000000000..fb064fbe47 --- /dev/null +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/FakeInputInitializer.java @@ -0,0 +1,43 @@ +package org.apache.tez.runtime.library; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.tez.runtime.api.Event; +import org.apache.tez.runtime.api.InputInitializer; +import org.apache.tez.runtime.api.InputInitializerContext; +import org.apache.tez.runtime.api.events.InputConfigureVertexTasksEvent; +import org.apache.tez.runtime.api.events.InputDataInformationEvent; +import org.apache.tez.runtime.api.events.InputInitializerEvent; + +public class FakeInputInitializer extends InputInitializer { + + private static final int srcParallelism = 1; + + /** + * Constructor an instance of the InputInitializer. Classes extending this to create a + * InputInitializer, must provide the same constructor so that Tez can create an instance of + * the class at runtime. + * + * @param initializerContext initializer context which can be used to access the payload, vertex + * properties, etc + */ + public FakeInputInitializer(InputInitializerContext initializerContext) { + super(initializerContext); + } + + @Override + public List initialize() throws Exception { + List list = new ArrayList<>(); + list.add(InputConfigureVertexTasksEvent.create(srcParallelism, null, null)); + for (int i = 0; i < srcParallelism; i++) { + list.add(InputDataInformationEvent.createWithObjectPayload(i, null)); + } + return list; + } + + @Override + public void handleInputInitializerEvent(List events) throws Exception { + + } +} diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/FakeOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/FakeOutput.java new file mode 100644 index 0000000000..e5ef6e5fae --- /dev/null +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/FakeOutput.java @@ -0,0 +1,58 @@ +package org.apache.tez.runtime.library; + +import java.io.IOException; +import java.util.List; + +import org.apache.tez.runtime.api.AbstractLogicalOutput; +import org.apache.tez.runtime.api.Event; +import org.apache.tez.runtime.api.OutputContext; +import org.apache.tez.runtime.api.Writer; +import org.apache.tez.runtime.library.api.KeyValueWriter; + +public class FakeOutput extends AbstractLogicalOutput { + + /** + * Constructor an instance of the LogicalOutput. Classes extending this one to create a + * LogicalOutput, must provide the same constructor so that Tez can create an instance of the + * class at runtime. + * + * @param outputContext the {@link OutputContext} which + * provides + * the Output with context information within the running task. + * @param numPhysicalOutputs the number of physical outputs that the logical output will + */ + public FakeOutput(OutputContext outputContext, int numPhysicalOutputs) { + super(outputContext, numPhysicalOutputs); + } + + @Override + public List initialize() throws Exception { + getContext().requestInitialMemory(0, null); + return null; + } + + @Override + public void handleEvents(List outputEvents) { + + } + + @Override + public List close() throws Exception { + return null; + } + + @Override + public void start() throws Exception { + + } + + @Override + public Writer getWriter() throws Exception { + return new KeyValueWriter() { + @Override + public void write(Object key, Object value) throws IOException { + System.out.println(key + " XXX " + value); + } + }; + } +} diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/FakeOutputCommitter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/FakeOutputCommitter.java new file mode 100644 index 0000000000..132335d74b --- /dev/null +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/FakeOutputCommitter.java @@ -0,0 +1,40 @@ +package org.apache.tez.runtime.library; + +import org.apache.tez.dag.api.client.VertexStatus; +import org.apache.tez.runtime.api.OutputCommitter; +import org.apache.tez.runtime.api.OutputCommitterContext; + +public class FakeOutputCommitter extends OutputCommitter { + + /** + * Constructor an instance of the OutputCommitter. Classes extending this to create a + * OutputCommitter, must provide the same constructor so that Tez can create an instance of + * the class at runtime. + * + * @param committerContext committer context which can be used to access the payload, vertex + * properties, etc + */ + public FakeOutputCommitter(OutputCommitterContext committerContext) { + super(committerContext); + } + + @Override + public void initialize() throws Exception { + + } + + @Override + public void setupOutput() throws Exception { + + } + + @Override + public void commitOutput() throws Exception { + + } + + @Override + public void abortOutput(VertexStatus.State finalState) throws Exception { + + } +} From 021a52bdaad186f82c5675b3c724920dceaef0c8 Mon Sep 17 00:00:00 2001 From: Laszlo Bodor Date: Wed, 10 Sep 2025 13:36:14 +0200 Subject: [PATCH 2/5] TEZ-4007: Introduce AmExtensions and Zookeeper-based FrameworkServices - checkstyle, spotbugs, javadoc improvements, refactor, test fixes --- pom.xml | 14 +- tez-api/findbugs-exclude.xml | 13 + tez-api/pom.xml | 1 - .../apache/tez/client/FrameworkClient.java | 14 +- .../java/org/apache/tez/client/TezClient.java | 3 - .../org/apache/tez/client/TezClientUtils.java | 44 ++-- .../org/apache/tez/client/TezYarnClient.java | 2 +- .../apache/tez/client/registry/AMRecord.java | 65 +++-- .../tez/client/registry/AMRegistry.java | 33 ++- .../tez/client/registry/AMRegistryClient.java | 57 ++++- .../zookeeper/ZkAMRegistryClient.java | 233 +++++++++-------- .../client/registry/zookeeper/ZkConfig.java | 53 +++- .../registry/zookeeper/ZkFrameworkClient.java | 77 ++++-- .../registry/zookeeper/package-info.java | 24 ++ .../apache/tez/dag/api/TezConfiguration.java | 36 ++- .../org/apache/tez/dag/api/TezConstants.java | 8 +- .../dag/api/client/rpc/DAGClientRPCImpl.java | 1 + .../ClientFrameworkService.java | 33 ++- .../tez/frameworkplugins/FrameworkMode.java | 26 +- .../frameworkplugins/FrameworkService.java | 16 +- .../tez/frameworkplugins/FrameworkUtils.java | 38 ++- .../tez/frameworkplugins/package-info.java | 24 ++ .../yarn/YarnClientFrameworkService.java | 35 +++ .../frameworkplugins/yarn/package-info.java | 24 ++ .../ZkStandaloneClientFrameworkService.java | 29 +++ ...eeperStandaloneClientFrameworkService.java | 26 -- .../zookeeper/package-info.java | 24 ++ .../org/apache/tez/client/TestTezClient.java | 11 +- .../apache/tez/client/TestTezClientUtils.java | 12 +- .../zookeeper/TestZkFrameworkClient.java | 242 ++++++++++++++++++ .../apache/tez/common/TezUtilsInternal.java | 8 +- tez-dag/findbugs-exclude.xml | 7 + tez-dag/pom.xml | 5 - .../tez/dag/api/client/DAGClientServer.java | 9 +- .../registry/zookeeper/ZkAMRegistry.java | 63 +++-- .../registry/zookeeper/package-info.java | 24 ++ .../org/apache/tez/dag/app/AppContext.java | 3 + .../org/apache/tez/dag/app/DAGAppMaster.java | 126 ++++----- .../org/apache/tez/dag/app/PluginManager.java | 5 + .../apache/tez/dag/app/RecoveryParser.java | 1 - .../tez/dag/app/TezTaskCommunicatorImpl.java | 3 +- .../apache/tez/dag/app/dag/impl/DAGImpl.java | 72 +++--- .../tez/dag/app/dag/impl/TaskAttemptImpl.java | 14 +- .../apache/tez/dag/app/dag/impl/TaskImpl.java | 5 +- .../app/launcher/TezLocalCacheManager.java | 6 +- .../tez/dag/app/rm/TaskSchedulerManager.java | 2 +- .../tez/frameworkplugins/AMExtensions.java | 92 +++++++ .../tez/frameworkplugins/AmExtensions.java | 63 ----- .../ServerFrameworkService.java | 21 +- .../TaskResourceException.java | 24 ++ .../tez/frameworkplugins/package-info.java | 21 ++ .../yarn/YarnServerFrameworkService.java | 120 +++++++++ .../frameworkplugins/yarn/package-info.java | 21 ++ ...ons.java => ZkStandaloneAMExtensions.java} | 72 +++--- .../ZkStandaloneServerFrameworkService.java | 55 ++++ ...eeperStandaloneServerFrameworkService.java | 39 --- .../zookeeper/package-info.java | 21 ++ .../zookeeper/TestZkAMRegistryClient.java | 197 ++++++++++++++ .../tez/dag/app/dag/impl/TestCommit.java | 4 + .../tez/dag/app/dag/impl/TestDAGImpl.java | 7 + .../tez/dag/app/dag/impl/TestDAGRecovery.java | 2 + .../tez/dag/app/dag/impl/TestVertexImpl2.java | 21 +- .../dag/app/dag/impl/TestVertexManager.java | 1 + .../dag/app/rm/TestTaskSchedulerHelpers.java | 1 + tez-examples/pom.xml | 4 + .../org/apache/tez/examples/ExampleBase.java | 40 --- .../tez/examples/StandaloneWordCount.java | 128 --------- .../examples/StandaloneWordCountDriver.java | 109 -------- .../apache/tez/examples/TezExampleBase.java | 9 +- .../apache/tez/mapreduce/SumProcessor.java | 40 --- .../apache/tez/mapreduce/TokenProcessor.java | 45 ---- .../apache/tez/runtime/library/FakeInput.java | 80 ------ .../runtime/library/FakeInputInitializer.java | 43 ---- .../tez/runtime/library/FakeOutput.java | 58 ----- .../runtime/library/FakeOutputCommitter.java | 40 --- .../tez/runtime/library/package-info.java | 24 ++ 76 files changed, 1738 insertions(+), 1235 deletions(-) create mode 100644 tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/package-info.java create mode 100644 tez-api/src/main/java/org/apache/tez/frameworkplugins/package-info.java create mode 100644 tez-api/src/main/java/org/apache/tez/frameworkplugins/yarn/YarnClientFrameworkService.java create mode 100644 tez-api/src/main/java/org/apache/tez/frameworkplugins/yarn/package-info.java create mode 100644 tez-api/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZkStandaloneClientFrameworkService.java delete mode 100644 tez-api/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZookeeperStandaloneClientFrameworkService.java create mode 100644 tez-api/src/main/java/org/apache/tez/frameworkplugins/zookeeper/package-info.java create mode 100644 tez-api/src/test/java/org/apache/tez/client/registry/zookeeper/TestZkFrameworkClient.java create mode 100644 tez-dag/src/main/java/org/apache/tez/dag/api/client/registry/zookeeper/package-info.java create mode 100644 tez-dag/src/main/java/org/apache/tez/frameworkplugins/AMExtensions.java delete mode 100644 tez-dag/src/main/java/org/apache/tez/frameworkplugins/AmExtensions.java create mode 100644 tez-dag/src/main/java/org/apache/tez/frameworkplugins/TaskResourceException.java create mode 100644 tez-dag/src/main/java/org/apache/tez/frameworkplugins/package-info.java create mode 100644 tez-dag/src/main/java/org/apache/tez/frameworkplugins/yarn/YarnServerFrameworkService.java create mode 100644 tez-dag/src/main/java/org/apache/tez/frameworkplugins/yarn/package-info.java rename tez-dag/src/main/java/org/apache/tez/frameworkplugins/zookeeper/{ZkStandaloneAmExtensions.java => ZkStandaloneAMExtensions.java} (53%) create mode 100644 tez-dag/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZkStandaloneServerFrameworkService.java delete mode 100644 tez-dag/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZookeeperStandaloneServerFrameworkService.java create mode 100644 tez-dag/src/main/java/org/apache/tez/frameworkplugins/zookeeper/package-info.java create mode 100644 tez-dag/src/test/java/org/apache/tez/client/registry/zookeeper/TestZkAMRegistryClient.java delete mode 100644 tez-examples/src/main/java/org/apache/tez/examples/ExampleBase.java delete mode 100644 tez-examples/src/main/java/org/apache/tez/examples/StandaloneWordCount.java delete mode 100644 tez-examples/src/main/java/org/apache/tez/examples/StandaloneWordCountDriver.java delete mode 100644 tez-mapreduce/src/main/java/org/apache/tez/mapreduce/SumProcessor.java delete mode 100644 tez-mapreduce/src/main/java/org/apache/tez/mapreduce/TokenProcessor.java delete mode 100644 tez-runtime-library/src/main/java/org/apache/tez/runtime/library/FakeInput.java delete mode 100644 tez-runtime-library/src/main/java/org/apache/tez/runtime/library/FakeInputInitializer.java delete mode 100644 tez-runtime-library/src/main/java/org/apache/tez/runtime/library/FakeOutput.java delete mode 100644 tez-runtime-library/src/main/java/org/apache/tez/runtime/library/FakeOutputCommitter.java create mode 100644 tez-runtime-library/src/main/java/org/apache/tez/runtime/library/package-info.java diff --git a/pom.xml b/pom.xml index fd68f22f20..1369e75b24 100644 --- a/pom.xml +++ b/pom.xml @@ -69,7 +69,7 @@ 2.16.0 2.6 ${user.home}/clover.license - 2.7.1 + 5.9.0 3.2.0 3.8.1 4.9.3 @@ -749,6 +749,18 @@ test-jar test + + org.apache.curator + curator-test + ${curator.version} + test + + + org.junit.jupiter + junit-jupiter-api + + + org.mockito mockito-core diff --git a/tez-api/findbugs-exclude.xml b/tez-api/findbugs-exclude.xml index de8f3824cd..4952d18c4e 100644 --- a/tez-api/findbugs-exclude.xml +++ b/tez-api/findbugs-exclude.xml @@ -151,4 +151,17 @@ + + + + + + + + + + + + + diff --git a/tez-api/pom.xml b/tez-api/pom.xml index b6495f693f..9e889bdd39 100644 --- a/tez-api/pom.xml +++ b/tez-api/pom.xml @@ -126,7 +126,6 @@ org.apache.curator curator-test - ${curator.version} test diff --git a/tez-api/src/main/java/org/apache/tez/client/FrameworkClient.java b/tez-api/src/main/java/org/apache/tez/client/FrameworkClient.java index f5037e7f2d..f36a499314 100644 --- a/tez-api/src/main/java/org/apache/tez/client/FrameworkClient.java +++ b/tez-api/src/main/java/org/apache/tez/client/FrameworkClient.java @@ -19,7 +19,6 @@ package org.apache.tez.client; import java.io.IOException; -import java.util.Optional; import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.conf.Configuration; @@ -27,7 +26,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; -import org.apache.hadoop.yarn.client.api.YarnClient; import org.apache.hadoop.yarn.client.api.YarnClientApplication; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.tez.common.RPCUtil; @@ -49,6 +47,7 @@ import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.SubmitDAGResponseProto; import org.apache.tez.frameworkplugins.ClientFrameworkService; import org.apache.tez.frameworkplugins.FrameworkUtils; +import org.apache.tez.frameworkplugins.yarn.YarnClientFrameworkService; import com.google.protobuf.ServiceException; @@ -60,10 +59,6 @@ public abstract class FrameworkClient { protected static final Logger LOG = LoggerFactory.getLogger(FrameworkClient.class); public static FrameworkClient createFrameworkClient(TezConfiguration tezConf) { - Optional pluginClient = - FrameworkUtils.get(ClientFrameworkService.class, tezConf) - .flatMap(framework -> framework.createOrGetFrameworkClient(tezConf)); - boolean isLocal = tezConf.getBoolean(TezConfiguration.TEZ_LOCAL_MODE, TezConfiguration.TEZ_LOCAL_MODE_DEFAULT); if (isLocal) { try { @@ -71,10 +66,11 @@ public static FrameworkClient createFrameworkClient(TezConfiguration tezConf) { } catch (TezReflectionException e) { throw new TezUncheckedException("Fail to create LocalClient", e); } - } else if (pluginClient.isPresent()) { - return pluginClient.get(); + } else { + ClientFrameworkService clientFrameworkService = FrameworkUtils.get(ClientFrameworkService.class, tezConf); + return clientFrameworkService == null ? new YarnClientFrameworkService().newFrameworkClient() + : clientFrameworkService.newFrameworkClient(); } - return new TezYarnClient(YarnClient.createYarnClient()); } /** diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClient.java b/tez-api/src/main/java/org/apache/tez/client/TezClient.java index 0e4167e323..8c16f6be8c 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClient.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClient.java @@ -672,8 +672,6 @@ private DAGClient submitDAGSession(DAG dag) throws TezException, IOException { } Map tezJarResources = getTezJarResources(sessionCredentials); - - DAGPlan dagPlan = TezClientUtils.prepareAndCreateDAGPlan(dag, amConfig, tezJarResources, usingTezArchiveDeploy, sessionCredentials, servicePluginsDescriptor, javaOptsChecker); @@ -688,7 +686,6 @@ private DAGClient submitDAGSession(DAG dag) throws TezException, IOException { // if request size exceeds maxSubmitDAGRequestSizeThroughIPC, we serialize them to HDFS SubmitDAGRequestProto request = requestBuilder.build(); - if (request.getSerializedSize() > maxSubmitDAGRequestSizeThroughIPC) { Path dagPlanPath = new Path(TezCommonUtils.getTezSystemStagingPath(amConfig.getTezConfiguration(), sessionAppId.toString()), TezConstants.TEZ_PB_PLAN_BINARY_NAME + diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java index 92e44f17bc..0a3c14fd69 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java @@ -149,8 +149,6 @@ private static Path getPath(String configUri) { } } - - /** * Setup LocalResource map for Tez jars based on provided Configuration * @@ -188,7 +186,7 @@ static boolean setupTezJarsLocalResources(TezConfiguration conf, + conf.get(TezConfiguration.TEZ_LIB_URIS_CLASSPATH)); usingTezArchive = addLocalResources(conf, tezJarUris, - tezJarResources, credentials); + tezJarResources, credentials); if (tezJarResources.isEmpty()) { throw new TezUncheckedException( @@ -265,8 +263,8 @@ private static boolean addLocalResources(Configuration conf, // Add URI fragment or just the filename Path name = new Path((null == u.getFragment()) - ? p.getName() - : u.getFragment()); + ? p.getName() + : u.getFragment()); if (name.isAbsolute()) { throw new IllegalArgumentException("Resource name must be " + "relative, not absolute: " + name @@ -579,7 +577,7 @@ public static ApplicationSubmissionContext createApplicationSubmissionContext( // don't overwrite existing conf, needed for TezClient.getClient() so existing containers have stable resource fingerprints if(!binaryConfPath.getFileSystem(tezConf).exists(binaryConfPath)) { ConfigurationProto finalConfProto = createFinalConfProtoForApp(tezConf, - servicePluginsDescriptor); + servicePluginsDescriptor); FSDataOutputStream amConfPBOutBinaryStream = null; try { @@ -620,12 +618,12 @@ public static ApplicationSubmissionContext createApplicationSubmissionContext( } LocalResource sessionJarsPBLRsrc = - TezClientUtils.createLocalResource(fs, - sessionJarsPath, LocalResourceType.FILE, - LocalResourceVisibility.APPLICATION); + TezClientUtils.createLocalResource(fs, + sessionJarsPath, LocalResourceType.FILE, + LocalResourceVisibility.APPLICATION); amLocalResources.put( - TezConstants.TEZ_AM_LOCAL_RESOURCES_PB_FILE_NAME, - sessionJarsPBLRsrc); + TezConstants.TEZ_AM_LOCAL_RESOURCES_PB_FILE_NAME, + sessionJarsPBLRsrc); String user = UserGroupInformation.getCurrentUser().getShortUserName(); ACLManager aclManager = new ACLManager(user, amConfig.getTezConfiguration()); @@ -657,9 +655,9 @@ public static ApplicationSubmissionContext createApplicationSubmissionContext( } amLocalResources.put(TezConstants.TEZ_PB_PLAN_BINARY_NAME, - TezClientUtils.createLocalResource(fs, - binaryPath, LocalResourceType.FILE, - LocalResourceVisibility.APPLICATION)); + TezClientUtils.createLocalResource(fs, + binaryPath, LocalResourceType.FILE, + LocalResourceVisibility.APPLICATION)); if (Level.DEBUG.isGreaterOrEqual(Level.toLevel(amLogLevel))) { Path textPath = localizeDagPlanAsText(dagPB, fs, amConfig, strAppId, tezSysStagingPath); @@ -690,7 +688,7 @@ public static ApplicationSubmissionContext createApplicationSubmissionContext( Collection tagsFromConf = amConfig.getTezConfiguration().getTrimmedStringCollection( - TezConfiguration.TEZ_APPLICATION_TAGS); + TezConfiguration.TEZ_APPLICATION_TAGS); appContext.setApplicationType(TezConstants.TEZ_APPLICATION_TYPE); if (tagsFromConf != null && !tagsFromConf.isEmpty()) { @@ -711,8 +709,8 @@ public static ApplicationSubmissionContext createApplicationSubmissionContext( appContext.setAMContainerSpec(amContainer); appContext.setMaxAppAttempts( - amConfig.getTezConfiguration().getInt(TezConfiguration.TEZ_AM_MAX_APP_ATTEMPTS, - TezConfiguration.TEZ_AM_MAX_APP_ATTEMPTS_DEFAULT)); + amConfig.getTezConfiguration().getInt(TezConfiguration.TEZ_AM_MAX_APP_ATTEMPTS, + TezConfiguration.TEZ_AM_MAX_APP_ATTEMPTS_DEFAULT)); return appContext; @@ -850,7 +848,7 @@ public static void addLog4jSystemProperties(String logLevel, } public static ConfigurationProto createFinalConfProtoForApp(Configuration amConf, - ServicePluginsDescriptor servicePluginsDescriptor) { + ServicePluginsDescriptor servicePluginsDescriptor) { assert amConf != null; ConfigurationProto.Builder builder = ConfigurationProto.newBuilder(); for (Entry entry : amConf) { @@ -951,7 +949,7 @@ static DAGClientAMProtocolBlockingPB getAMProxy(FrameworkClient frameworkClient, + ", trackingUrl=" + appReport.getTrackingUrl() + ", diagnostics=" + (appReport.getDiagnostics() != null ? appReport.getDiagnostics() - : TezClient.NO_CLUSTER_DIAGNOSTICS_MSG); + : TezClient.NO_CLUSTER_DIAGNOSTICS_MSG); LOG.info(msg); throw new SessionNotRunning(msg); } @@ -1017,7 +1015,7 @@ static void createSessionToken(String tokenIdentifier, public static String maybeAddDefaultMemoryJavaOpts(String javaOpts, Resource resource, double maxHeapFactor) { if ((javaOpts != null && !javaOpts.isEmpty() - && (javaOpts.contains("-Xmx") || javaOpts.contains("-Xms"))) + && (javaOpts.contains("-Xmx") || javaOpts.contains("-Xms"))) || (resource.getMemory() <= 0)) { return javaOpts; } @@ -1028,8 +1026,8 @@ public static String maybeAddDefaultMemoryJavaOpts(String javaOpts, Resource res if (Double.parseDouble("-1") == maxHeapFactor) { maxHeapFactor = resource.getMemory() < TezConstants.TEZ_CONTAINER_SMALL_SLAB_BOUND_MB - ? TezConstants.TEZ_CONTAINER_MAX_JAVA_HEAP_FRACTION_SMALL_SLAB - : TezConstants.TEZ_CONTAINER_MAX_JAVA_HEAP_FRACTION_LARGE_SLAB; + ? TezConstants.TEZ_CONTAINER_MAX_JAVA_HEAP_FRACTION_SMALL_SLAB + : TezConstants.TEZ_CONTAINER_MAX_JAVA_HEAP_FRACTION_LARGE_SLAB; } int maxMemory = (int)(resource.getMemory() * maxHeapFactor); maxMemory = maxMemory <= 0 ? 1 : maxMemory; @@ -1039,7 +1037,7 @@ public static String maybeAddDefaultMemoryJavaOpts(String javaOpts, Resource res } private static boolean checkAncestorPermissionsForAllUsers(Configuration conf, Path pathComponent, - FsAction permission) throws IOException { + FsAction permission) throws IOException { FileSystem fs = pathComponent.getFileSystem(conf); if (Shell.WINDOWS && fs instanceof LocalFileSystem) { diff --git a/tez-api/src/main/java/org/apache/tez/client/TezYarnClient.java b/tez-api/src/main/java/org/apache/tez/client/TezYarnClient.java index d109648075..eed5129610 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezYarnClient.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezYarnClient.java @@ -43,7 +43,7 @@ public class TezYarnClient extends FrameworkClient { private String amHost; private int amPort; - protected TezYarnClient(YarnClient yarnClient) { + public TezYarnClient(YarnClient yarnClient) { this.yarnClient = yarnClient; } diff --git a/tez-api/src/main/java/org/apache/tez/client/registry/AMRecord.java b/tez-api/src/main/java/org/apache/tez/client/registry/AMRecord.java index 2f5ab978d5..c1c2608883 100644 --- a/tez-api/src/main/java/org/apache/tez/client/registry/AMRecord.java +++ b/tez-api/src/main/java/org/apache/tez/client/registry/AMRecord.java @@ -23,6 +23,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.registry.client.types.ServiceRecord; import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.tez.client.registry.zookeeper.ZkConfig; /** @@ -37,14 +38,18 @@ @InterfaceAudience.Public public class AMRecord { private static final String APP_ID_RECORD_KEY = "appId"; - private static final String HOST_RECORD_KEY = "host"; + private static final String HOST_NAME_RECORD_KEY = "hostName"; + private static final String HOST_IP_RECORD_KEY = "hostIp"; private static final String PORT_RECORD_KEY = "port"; private static final String EXTERNAL_ID_KEY = "externalId"; + private static final String COMPUTE_GROUP_NAME_KEY = "computeName"; private final ApplicationId appId; - private final String host; + private final String hostName; + private final String hostIp; private final int port; private final String externalId; + private final String computeName; /** * Creates a new {@code AMRecord} with the given application ID, host, port, and identifier. @@ -55,16 +60,20 @@ public class AMRecord { * it is part of the public API for Tez clients that handle unmanaged sessions. * * @param appId the {@link ApplicationId} of the Tez application - * @param host the hostname where the Application Master is running - * @param port the port number on which the Application Master is listening - * @param externalId an opaque identifier for the record; if {@code null}, defaults to an empty string + * @param hostName the hostname where the Application Master is running + * @param hostIp the IP address of the Application Master host + * @param port the RPC port number on which the Application Master is listening + * @param externalId an optional external identifier for the record; if {@code null}, defaults to an empty string + * @param computeName the compute group or cluster name; if {@code null}, defaults to {@link ZkConfig#DEFAULT_COMPUTE_GROUP_NAME} */ - public AMRecord(ApplicationId appId, String host, int port, String externalId) { + public AMRecord(ApplicationId appId, String hostName, String hostIp, int port, String externalId, String computeName) { this.appId = appId; - this.host = host; + this.hostName = hostName; + this.hostIp = hostIp; this.port = port; //externalId is optional, if not provided, convert to empty string this.externalId = (externalId == null) ? "" : externalId; + this.computeName = (computeName == null) ? ZkConfig.DEFAULT_COMPUTE_GROUP_NAME : computeName; } /** @@ -79,9 +88,11 @@ public AMRecord(ApplicationId appId, String host, int port, String externalId) { */ public AMRecord(AMRecord other) { this.appId = other.getApplicationId(); - this.host = other.getHost(); + this.hostName = other.getHost(); + this.hostIp = other.getHostIp(); this.port = other.getPort(); this.externalId = other.getExternalId(); + this.computeName = other.getComputeName(); } /** @@ -97,9 +108,11 @@ public AMRecord(AMRecord other) { */ public AMRecord(ServiceRecord serviceRecord) { this.appId = ApplicationId.fromString(serviceRecord.get(APP_ID_RECORD_KEY)); - this.host = serviceRecord.get(HOST_RECORD_KEY); + this.hostName = serviceRecord.get(HOST_NAME_RECORD_KEY); + this.hostIp = serviceRecord.get(HOST_IP_RECORD_KEY); this.port = Integer.parseInt(serviceRecord.get(PORT_RECORD_KEY)); this.externalId = serviceRecord.get(EXTERNAL_ID_KEY); + this.computeName = serviceRecord.get(COMPUTE_GROUP_NAME_KEY); } public ApplicationId getApplicationId() { @@ -107,7 +120,15 @@ public ApplicationId getApplicationId() { } public String getHost() { - return host; + return hostName; + } + + public String getHostName() { + return hostName; + } + + public String getHostIp() { + return hostIp; } public int getPort() { @@ -118,16 +139,23 @@ public String getExternalId() { return externalId; } + public String getComputeName() { + return computeName; + } + @Override public boolean equals(Object other) { if (this == other) { return true; } - if (other instanceof AMRecord otherRecord) { + if (other instanceof AMRecord) { + AMRecord otherRecord = (AMRecord) other; return appId.equals(otherRecord.appId) - && host.equals(otherRecord.host) + && hostName.equals(otherRecord.hostName) + && hostIp.equals(otherRecord.hostIp) && port == otherRecord.port - && externalId.equals(otherRecord.externalId); + && externalId.equals(otherRecord.externalId) + && computeName.equals(otherRecord.computeName); } else { return false; } @@ -150,14 +178,21 @@ public boolean equals(Object other) { public ServiceRecord toServiceRecord() { ServiceRecord serviceRecord = new ServiceRecord(); serviceRecord.set(APP_ID_RECORD_KEY, appId); - serviceRecord.set(HOST_RECORD_KEY, host); + serviceRecord.set(HOST_NAME_RECORD_KEY, hostName); + serviceRecord.set(HOST_IP_RECORD_KEY, hostIp); serviceRecord.set(PORT_RECORD_KEY, port); serviceRecord.set(EXTERNAL_ID_KEY, externalId); + serviceRecord.set(COMPUTE_GROUP_NAME_KEY, computeName); return serviceRecord; } + @Override + public String toString() { + return toServiceRecord().attributes().toString(); + } + @Override public int hashCode() { - return Objects.hash(appId, host, port, externalId); + return Objects.hash(appId, hostName, hostIp, externalId, computeName, port); } } diff --git a/tez-api/src/main/java/org/apache/tez/client/registry/AMRegistry.java b/tez-api/src/main/java/org/apache/tez/client/registry/AMRegistry.java index bb1b16da7c..e5f96b1f80 100644 --- a/tez-api/src/main/java/org/apache/tez/client/registry/AMRegistry.java +++ b/tez-api/src/main/java/org/apache/tez/client/registry/AMRegistry.java @@ -20,7 +20,6 @@ import java.util.ArrayList; import java.util.List; -import java.util.Optional; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.service.AbstractService; @@ -31,19 +30,28 @@ import org.slf4j.LoggerFactory; /** - * Base class for AMRegistry implementation - * Implementation class is configured by tez.am.registry.class - * Implementations should implement relevant service lifecycle operations: - * init, serviceStart, serviceStop, etc.. + * Base class for {@code AMRegistry} implementations. * - * init/serviceStart will be invoked during DAGAppMaster.serviceInit + *

    The specific implementation is configured via the + * {@code tez.am.registry.class} property.

    * - * serviceStop will invoked on DAGAppMaster shutdown + *

    Implementations are expected to provide appropriate service lifecycle + * behavior, including: + *

      + *
    • {@code init}
    • + *
    • {@code serviceStart}
    • + *
    • {@code serviceStop}
    • + *
    + *

    + * + *

    {@code init} and {@code serviceStart} are invoked during + * {@code DAGAppMaster.serviceInit()}, while {@code serviceStop} is called + * when {@code DAGAppMaster} shuts down.

    */ public abstract class AMRegistry extends AbstractService { private static final Logger LOG = LoggerFactory.getLogger(AMRegistry.class); - protected List amRecords = new ArrayList<>(); + private List amRecords = new ArrayList<>(); @Override public void init(Configuration conf) { @@ -79,13 +87,14 @@ public void add(AMRecord server) throws Exception { public abstract void remove(AMRecord server) throws Exception; - public Optional generateNewId() throws Exception { - return Optional.empty(); + public ApplicationId generateNewId() throws Exception { + return null; } - public abstract AMRecord createAmRecord(ApplicationId appId, String hostName, int port); + public abstract AMRecord createAmRecord(ApplicationId appId, String hostName, String hostIp, int port, String computeName); - @Override public void serviceStop() throws Exception { + @Override + public void serviceStop() throws Exception { List records = new ArrayList<>(amRecords); for(AMRecord record : records) { remove(record); diff --git a/tez-api/src/main/java/org/apache/tez/client/registry/AMRegistryClient.java b/tez-api/src/main/java/org/apache/tez/client/registry/AMRegistryClient.java index 9a34fbb78e..e3cd1dfe04 100644 --- a/tez-api/src/main/java/org/apache/tez/client/registry/AMRegistryClient.java +++ b/tez-api/src/main/java/org/apache/tez/client/registry/AMRegistryClient.java @@ -24,32 +24,75 @@ import java.util.List; /** - * Interface for client-side AM discovery + * Client-side interface for discovering Application Master (AM) instances + * registered in the AM registry. + * + *

    Implementations are responsible for locating AM endpoints and returning + * their metadata. This API is used by client components to discover running + * Tez AMs.

    + * + *

    Listeners may be registered to receive notifications when AM records + * appear or are removed.

    */ public abstract class AMRegistryClient implements Closeable { - protected List listeners = new ArrayList<>(); + private final List listeners = new ArrayList<>(); - //Get AM info given an appId + /** + * Returns the current set of registered listeners. + * + * @return a mutable list of listeners + */ + protected List getListeners() { + return listeners; + } + + /** + * Lookup AM metadata for the given application ID. + * + * @param appId the application ID + * @return the AM record if found, otherwise {@code null} + * @throws IOException if the lookup fails + */ public abstract AMRecord getRecord(String appId) throws IOException; - //Get all AM infos in the registry + /** + * Retrieve all AM records known in the registry. + * + * @return a list of AM records (possibly empty) + * @throws IOException if the fetch fails + */ public abstract List getAllRecords() throws IOException; + /** + * Register a listener for AM registry events. + * The listener will be notified when AM records are added or removed. + * + * @param listener the listener to add + */ public synchronized void addListener(AMRegistryClientListener listener) { listeners.add(listener); } + /** + * Notify listeners of a newly added AM record. + * + * @param record the added AM record + */ protected synchronized void notifyOnAdded(AMRecord record) { - for(AMRegistryClientListener listener : listeners) { + for (AMRegistryClientListener listener : listeners) { listener.onAdd(record); } } + /** + * Notify listeners of a removed AM record. + * + * @param record the removed AM record + */ protected synchronized void notifyOnRemoved(AMRecord record) { - for(AMRegistryClientListener listener : listeners) { + for (AMRegistryClientListener listener : listeners) { listener.onRemove(record); } } - } diff --git a/tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/ZkAMRegistryClient.java b/tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/ZkAMRegistryClient.java index 41ce21eb31..dc7d9334dd 100644 --- a/tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/ZkAMRegistryClient.java +++ b/tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/ZkAMRegistryClient.java @@ -1,20 +1,20 @@ /** - * 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. -*/ + * 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.tez.client.registry.zookeeper; @@ -28,38 +28,42 @@ import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.imps.CuratorFrameworkState; import org.apache.curator.framework.recipes.cache.ChildData; -import org.apache.curator.framework.recipes.cache.PathChildrenCache; -import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent; -import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener; +import org.apache.curator.framework.recipes.cache.TreeCache; +import org.apache.curator.framework.recipes.cache.TreeCacheEvent; +import org.apache.curator.framework.recipes.cache.TreeCacheListener; +import org.apache.curator.shaded.com.google.common.base.Charsets; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.registry.client.binding.RegistryUtils; import org.apache.hadoop.registry.client.types.ServiceRecord; +import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.tez.client.registry.AMRecord; import org.apache.tez.client.registry.AMRegistryClient; import org.apache.tez.client.registry.AMRegistryClientListener; import org.apache.tez.dag.api.TezConfiguration; +import com.fasterxml.jackson.core.JsonParseException; import com.google.common.base.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** - * Curator/Zookeeper impl of AMRegistryClient -*/ + * Curator/Zookeeper implementation of {@link AMRegistryClient}. + */ @InterfaceAudience.Public -public class ZkAMRegistryClient extends AMRegistryClient { +public final class ZkAMRegistryClient extends AMRegistryClient { private static final Logger LOG = LoggerFactory.getLogger(ZkAMRegistryClient.class); + private static final Map INSTANCES = new HashMap<>(); private final Configuration conf; - //Cache of known AMs - private ConcurrentHashMap amRecordCache = new ConcurrentHashMap<>(); + private final ConcurrentHashMap amRecordCache = new ConcurrentHashMap<>(); private CuratorFramework client; - private PathChildrenCache cache; - private static Map INSTANCES = new HashMap<>(); + private ZkAMRegistryClient(final Configuration conf) { + this.conf = conf; + } public static synchronized ZkAMRegistryClient getClient(final Configuration conf) { String namespace = conf.get(TezConfiguration.TEZ_AM_REGISTRY_NAMESPACE); @@ -72,39 +76,48 @@ public static synchronized ZkAMRegistryClient getClient(final Configuration conf return registry; } - private ZkAMRegistryClient(final Configuration conf) { - this.conf = conf; + /** + * Deserializes a {@link ServiceRecord} from ZooKeeper data and converts it into an {@link AMRecord} + * for caching. + * + * @param childData the ZooKeeper node data containing a serialized {@link ServiceRecord} + * @return an {@link AMRecord} constructed from the deserialized {@link ServiceRecord}, or {@code null} + * if no data is present + * @throws IOException if the data cannot be deserialized into a {@link ServiceRecord} + */ + public static AMRecord getAMRecord(final ChildData childData) throws IOException { + // not a leaf path. Only leaf path contains AMRecord + if (!childData.getPath().contains(ApplicationId.appIdStrPrefix)) { + return null; + } + byte[] data = childData.getData(); + // only the path appeared, there is no data yet + if (data.length == 0) { + return null; + } + String value = new String(data, Charsets.UTF_8); + try { + RegistryUtils.ServiceRecordMarshal marshal = new RegistryUtils.ServiceRecordMarshal(); + ServiceRecord serviceRecord = marshal.fromJson(value); + return new AMRecord(serviceRecord); + } catch (JsonParseException e) { + //Not a json AMRecord (SRV), could be some other data. + LOG.warn("Non-json data received while de-serializing AMRecord: {}. Ignoring...", value); + return null; + } } public void start() throws Exception { ZkConfig zkConf = new ZkConfig(this.conf); client = zkConf.createCuratorFramework(); - cache = new PathChildrenCache(client, zkConf.getZkNamespace(), true); + final TreeCache cache = new TreeCache(client, zkConf.getZkNamespace()); client.start(); - cache.start(PathChildrenCache.StartMode.BUILD_INITIAL_CACHE); - for (ChildData childData : cache.getCurrentData()) { - AMRecord amRecord = getAMRecord(childData); - if (amRecord != null) { - amRecordCache.put(amRecord.getApplicationId().toString(), amRecord); - } - } + cache.start(); cache.getListenable().addListener(new ZkRegistryListener()); } - //Deserialize ServiceRecord from Zookeeper to populate AMRecord in cache - public static AMRecord getAMRecord(final ChildData childData) throws IOException { - byte[] data = childData.getData(); - // only the path appeared, there is no data yet - if (data.length == 0) { - return null; - } - String value = new String(data); - RegistryUtils.ServiceRecordMarshal marshal = new RegistryUtils.ServiceRecordMarshal(); - ServiceRecord serviceRecord = marshal.fromJson(value); - return new AMRecord(serviceRecord); - } - - @Override public AMRecord getRecord(String appId) { + @Override + public AMRecord getRecord(String appId) { if (amRecordCache.get(appId) == null) { return null; } @@ -112,67 +125,76 @@ public static AMRecord getAMRecord(final ChildData childData) throws IOException return new AMRecord(amRecordCache.get(appId)); } - @Override public List getAllRecords() { - return amRecordCache.values().stream() - .map(record -> new AMRecord(record)).collect(Collectors.toList()); - } + @Override + public List getAllRecords() { + return amRecordCache.values().stream().map(AMRecord::new).collect(Collectors.toList()); + } - @Override public synchronized void addListener(AMRegistryClientListener listener) { - listeners.add(listener); + @Override + public synchronized void addListener(AMRegistryClientListener listener) { + getListeners().add(listener); } - //Callback for Zookeeper to update local cache - private class ZkRegistryListener implements PathChildrenCacheListener { + @Override + public void close() { + client.close(); + } - @Override public void childEvent(final CuratorFramework client, final PathChildrenCacheEvent event) - throws Exception { - Preconditions.checkArgument(client != null && client.getState() == CuratorFrameworkState.STARTED, + /** + * Callback listener for ZooKeeper events that updates the local cache + * when child nodes under the monitored path change. + */ + private class ZkRegistryListener implements TreeCacheListener { + + @Override + public void childEvent(final CuratorFramework clientParam, final TreeCacheEvent event) throws Exception { + Preconditions.checkArgument(clientParam != null && clientParam.getState() == CuratorFrameworkState.STARTED, "Curator client is not started"); - ChildData childData = event.getData(); - switch (event.getType()) { - case CHILD_ADDED: - if(isEmpty(childData)) { - LOG.info("AppId allocated: {}", childData.getPath()); - } else { - AMRecord amRecord = getAMRecord(childData); - if (amRecord != null) { - LOG.info("AM registered with data: {}. Notifying {} listeners.", amRecord, listeners.size()); - amRecordCache.put(amRecord.getApplicationId().toString(), amRecord); - notifyOnAdded(amRecord); + ChildData childData = event.getData(); + switch (event.getType()) { + case NODE_ADDED: + if (isEmpty(childData)) { + LOG.info("AppId allocated: {}", childData.getPath()); + } else { + AMRecord amRecord = getAMRecord(childData); + if (amRecord != null) { + LOG.info("AM registered with data: {}. Notifying {} listeners.", amRecord, getListeners().size()); + amRecordCache.put(amRecord.getApplicationId().toString(), amRecord); + notifyOnAdded(amRecord); + } + } + break; + case NODE_UPDATED: + if (isEmpty(childData)) { + throw new RuntimeException("AM updated with empty data"); + } else { + AMRecord amRecord = getAMRecord(childData); + if (amRecord != null) { + LOG.info("AM updated data: {}. Notifying {} listeners.", amRecord, getListeners().size()); + amRecordCache.put(amRecord.getApplicationId().toString(), amRecord); + notifyOnAdded(amRecord); + } } - } - break; - case CHILD_UPDATED: - if(isEmpty(childData)) { - throw new RuntimeException("AM updated with empty data"); - } else { - AMRecord amRecord = getAMRecord(childData); - if (amRecord != null) { - LOG.info("AM updated data: {}. Notifying {} listeners.", amRecord, listeners.size()); - amRecordCache.put(amRecord.getApplicationId().toString(), amRecord); - notifyOnAdded(amRecord); + break; + case NODE_REMOVED: + if (isEmpty(childData)) { + LOG.info("Unused AppId unregistered: {}", childData.getPath()); + } else { + AMRecord amRecord = getAMRecord(childData); + if (amRecord != null) { + LOG.info("AM removed: {}. Notifying {} listeners.", amRecord, getListeners().size()); + amRecordCache.remove(amRecord.getApplicationId().toString(), amRecord); + notifyOnRemoved(amRecord); + } } - } - break; - case CHILD_REMOVED: - if(isEmpty(childData)) { - LOG.info("Unused AppId unregistered: {}", childData.getPath()); - } else { - AMRecord amRecord = getAMRecord(childData); - if (amRecord != null) { - LOG.info("AM removed: {}. Notifying {} listeners.", amRecord, listeners.size()); - amRecordCache.remove(amRecord.getApplicationId().toString(), amRecord); - notifyOnRemoved(amRecord); + break; + default: + if (childData == null) { + LOG.info("Ignored event {}", event.getType()); + } else { + LOG.info("Ignored event {} for {}", event.getType(), childData.getPath()); } - } - break; - default: - if(childData == null) { - LOG.info("Ignored event {}", event.getType()); - } else { - LOG.info("Ignored event {} for {}", event.getType(), childData.getPath()); - } } } @@ -180,9 +202,4 @@ private boolean isEmpty(ChildData childData) { return childData == null || childData.getData() == null || childData.getData().length == 0; } } - - @Override - public void close() { - client.close(); - } -} \ No newline at end of file +} diff --git a/tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/ZkConfig.java b/tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/ZkConfig.java index 38d32838b6..312c4b1453 100644 --- a/tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/ZkConfig.java +++ b/tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/ZkConfig.java @@ -23,25 +23,62 @@ import org.apache.curator.framework.CuratorFrameworkFactory; import org.apache.curator.retry.ExponentialBackoffRetry; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; import org.apache.tez.dag.api.TezConfiguration; import com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + public class ZkConfig { + private static final Logger LOG = LoggerFactory.getLogger(ZkConfig.class); + + // if namespace defined in config is 'foo' and COMPUTE_GROUP_NAME env is 'bar' then the zkpaths will be of format + // /tez-external-sessions/foo/bar + private final static String ZK_NAMESPACE_PREFIX = "/tez-external-sessions"; + public final static String COMPUTE_GROUP_NAME_ENV = "COMPUTE_GROUP_NAME"; + public final static String DEFAULT_COMPUTE_GROUP_NAME = "default-compute"; - private String zkQuorum; - private String zkNamespace; - private int curatorBackoffSleep; - private int curatorMaxRetries; - private int sessionTimeoutMs; - private int connectionTimeoutMs; + private final String zkQuorum; + private final String zkNamespace; + private final int curatorBackoffSleep; + private final int curatorMaxRetries; + private final int sessionTimeoutMs; + private final int connectionTimeoutMs; public ZkConfig(Configuration conf) { zkQuorum = conf.get(TezConfiguration.TEZ_AM_ZOOKEEPER_QUORUM); Preconditions.checkNotNull(zkQuorum); - zkNamespace = conf.get(TezConfiguration.TEZ_AM_REGISTRY_NAMESPACE, + + String fullZkNamespace = ZK_NAMESPACE_PREFIX; + + String namespace = conf.get(TezConfiguration.TEZ_AM_REGISTRY_NAMESPACE, TezConfiguration.TEZ_AM_REGISTRY_NAMESPACE_DEFAULT); - Preconditions.checkNotNull(zkNamespace); + Preconditions.checkNotNull(namespace); + if (namespace.startsWith(Path.SEPARATOR)) { + fullZkNamespace += namespace; + } else { + fullZkNamespace = fullZkNamespace + Path.SEPARATOR + namespace; + } + + boolean enableComputeGroups = conf.getBoolean(TezConfiguration.TEZ_AM_REGISTRY_ENABLE_COMPUTE_GROUPS, + TezConfiguration.TEZ_AM_REGISTRY_ENABLE_COMPUTE_GROUPS_DEFAULT); + if (enableComputeGroups) { + final String subNamespace = System.getenv(COMPUTE_GROUP_NAME_ENV); + if (subNamespace != null && !subNamespace.isEmpty()) { + if (subNamespace.startsWith(Path.SEPARATOR)) { + fullZkNamespace += subNamespace; + } else { + fullZkNamespace = fullZkNamespace + Path.SEPARATOR + subNamespace; + } + LOG.info("Compute groups enabled: subNamespace: {} fullZkNamespace: {}", subNamespace, fullZkNamespace); + } + } else { + LOG.info("Compute groups disabled: fullZkNamespace: {}", fullZkNamespace); + } + zkNamespace = fullZkNamespace; + curatorBackoffSleep = conf.getInt(TezConfiguration.TEZ_AM_CURATOR_BACKOFF_SLEEP, TezConfiguration.TEZ_AM_CURATOR_BACKOFF_SLEEP_DEFAULT); curatorMaxRetries = conf.getInt(TezConfiguration.TEZ_AM_CURATOR_MAX_RETRIES, diff --git a/tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/ZkFrameworkClient.java b/tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/ZkFrameworkClient.java index bf133c67c0..d743defcb5 100644 --- a/tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/ZkFrameworkClient.java +++ b/tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/ZkFrameworkClient.java @@ -6,9 +6,9 @@ * 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 - * + *

    + * 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. @@ -32,20 +32,17 @@ import org.apache.tez.client.FrameworkClient; import org.apache.tez.client.registry.AMRecord; import org.apache.tez.dag.api.TezConfiguration; -import org.apache.tez.dag.api.TezException; public class ZkFrameworkClient extends FrameworkClient { private AMRecord amRecord; - private TezConfiguration tezConf; private ZkAMRegistryClient amRegistryClient = null; - private boolean isRunning = false; + private volatile boolean isRunning = false; private String amHost; private int amPort; @Override public synchronized void init(TezConfiguration tezConf) { - this.tezConf = tezConf; if (this.amRegistryClient == null) { try { this.amRegistryClient = ZkAMRegistryClient.getClient(tezConf); @@ -56,25 +53,61 @@ public synchronized void init(TezConfiguration tezConf) { } } - @Override public void start() { + @Override + public void start() { try { this.amRegistryClient.start(); + isRunning = true; } catch (Exception e) { throw new RuntimeException(e); } - isRunning = true; } - @Override public void stop() { + @Override + public void stop() { isRunning = false; - amRegistryClient.close(); + close(); } - @Override public void close() throws IOException { - amRegistryClient.close(); + @Override + public void close() { + if (amRegistryClient != null) { + amRegistryClient.close(); + } } - @Override public YarnClientApplication createApplication() throws YarnException, IOException { + /** + * Creates a dummy {@link YarnClientApplication} using a pre-existing {@link ApplicationId} + * rather than requesting a new one from the ResourceManager. + * + *

    Note: This is a dummy, backward-compatibility implementation. + * Instead of allocating a fresh application ID from the ResourceManager, this method + * reuses the {@code applicationId} already obtained via {@code getApplicationReport()}. + * This allows legacy code paths to continue operating without requiring actual + * creation of a new application.

    + * + *

    Hidden assumption here: this method assumes that + * {@code getApplicationReport()} has already been called before + * {@code createApplication()}, ensuring that {@code amRecord.getApplicationId()} + * is always available. This assumption holds in all supported usage patterns: + * the only code path where {@code createApplication()} might be called first is + * {@code TezClient.submitDAGApplication()}, but that path is never exercised in + * Zookeeper standalone mode because that mode assumes applications are already + * running. Therefore, the ordering guarantee is valid in practice.

    + * + *

    + * The method constructs a minimal {@link ApplicationSubmissionContext} and a + * synthetic {@link GetNewApplicationResponse}, both populated with the already + * known application ID. These objects are then wrapped into a + * {@link YarnClientApplication} instance and returned. + *

    + * + * @return a {@link YarnClientApplication} backed by a submission context and + * a mocked {@link GetNewApplicationResponse}, both tied to the pre-existing + * application ID. + */ + @Override + public YarnClientApplication createApplication() { ApplicationSubmissionContext context = Records.newRecord(ApplicationSubmissionContext.class); ApplicationId appId = amRecord.getApplicationId(); context.setApplicationId(appId); @@ -83,17 +116,18 @@ public synchronized void init(TezConfiguration tezConf) { return new YarnClientApplication(response, context); } - @Override public ApplicationId submitApplication(ApplicationSubmissionContext appSubmissionContext) - throws YarnException, IOException, TezException { - //Unused + @Override + public ApplicationId submitApplication(ApplicationSubmissionContext appSubmissionContext) { return null; } - @Override public void killApplication(ApplicationId appId) throws YarnException, IOException { - amRegistryClient.close(); + @Override + public void killApplication(ApplicationId appId) throws YarnException, IOException { + close(); } - @Override public ApplicationReport getApplicationReport(ApplicationId appId) throws YarnException, IOException { + @Override + public ApplicationReport getApplicationReport(ApplicationId appId) throws YarnException, IOException { ApplicationReport report = Records.newRecord(ApplicationReport.class); report.setApplicationId(appId); report.setTrackingUrl(""); @@ -113,7 +147,8 @@ public synchronized void init(TezConfiguration tezConf) { return report; } - @Override public boolean isRunning() throws IOException { + @Override + public boolean isRunning() { return isRunning; } diff --git a/tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/package-info.java b/tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/package-info.java new file mode 100644 index 0000000000..ea4412da84 --- /dev/null +++ b/tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/package-info.java @@ -0,0 +1,24 @@ +/** + * 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. + */ + +@Public +@Evolving +package org.apache.tez.client.registry.zookeeper; + +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Evolving; diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java index 6caed299a7..5de35bdd82 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java @@ -869,8 +869,7 @@ public TezConfiguration(boolean loadDefaults) { /** Int value. Port used for AM RPC*/ @ConfigurationScope(Scope.AM) @ConfigurationProperty(type="integer") - public static final String TEZ_AM_RPC_PORT = - TEZ_AM_PREFIX + "rpc.port"; + public static final String TEZ_AM_RPC_PORT = TEZ_AM_PREFIX + "rpc.port"; public static final int TEZ_AM_RPC_PORT_DEFAULT = 0; /** Int value. Number of threads to handle client RPC requests. Expert level setting.*/ @@ -2351,14 +2350,14 @@ static Set getPropertySet() { public static final String TEZ_TASK_ATTEMPT_HOOKS = TEZ_TASK_PREFIX + "attempt.hooks"; /** - * String value + * String value. ZooKeeper quorum connection string used when creating a CuratorFramework for the ZooKeeper registry. */ @ConfigurationScope(Scope.AM) @ConfigurationProperty public static final String TEZ_AM_ZOOKEEPER_QUORUM = TEZ_AM_PREFIX + "zookeeper.quorum"; /** - * String value + * String value. Namespace in ZooKeeper registry for the Application Master. */ @ConfigurationScope(Scope.AM) @ConfigurationProperty @@ -2366,7 +2365,17 @@ static Set getPropertySet() { public static final String TEZ_AM_REGISTRY_NAMESPACE_DEFAULT = "/tez_am/server"; /** - * Integer value + * Boolean value. Whether to enable compute groups, see further details in ZkConfig. + */ + @ConfigurationScope(Scope.AM) + @ConfigurationProperty + public static final String TEZ_AM_REGISTRY_ENABLE_COMPUTE_GROUPS = TEZ_AM_PREFIX + "registry.enable.compute.groups"; + public static final boolean TEZ_AM_REGISTRY_ENABLE_COMPUTE_GROUPS_DEFAULT = false; + + + /** + * Integer value. Initial backoff sleep duration (in milliseconds) for Curator retries. + * It's used when creating a CuratorFramework for the ZooKeeper registry. */ @ConfigurationScope(Scope.AM) @ConfigurationProperty @@ -2374,7 +2383,8 @@ static Set getPropertySet() { public static final int TEZ_AM_CURATOR_BACKOFF_SLEEP_DEFAULT = 1000; /** - * Integer value + * Integer value. Maximum number of retries for Curator operations. + * It's used when creating a CuratorFramework for the ZooKeeper registry. */ @ConfigurationScope(Scope.AM) @ConfigurationProperty @@ -2382,26 +2392,30 @@ static Set getPropertySet() { public static final int TEZ_AM_CURATOR_MAX_RETRIES_DEFAULT = 3; /** - * Integer value (milliseconds) + * Integer value. Session timeout (in milliseconds) for Curator framework. + * It's used when creating a CuratorFramework for the ZooKeeper registry. */ @ConfigurationScope(Scope.AM) @ConfigurationProperty public static final String TEZ_AM_CURATOR_SESSION_TIMEOUT = TEZ_AM_PREFIX + "curator.session.timeout"; - public static final int TEZ_AM_CURATOR_SESSION_TIMEOUT_DEFAULT = 60000; + public static final int TEZ_AM_CURATOR_SESSION_TIMEOUT_DEFAULT = 150000; /** - * Integer value (milliseconds) - */ + * Integer value. Connection timeout (in milliseconds) for Curator framework. + * It's used when creating a CuratorFramework for the ZooKeeper registry. + */ @ConfigurationScope(Scope.AM) @ConfigurationProperty public static final String TEZ_AM_CURATOR_CONNECTION_TIMEOUT = TEZ_AM_PREFIX + "curator.connection.timeout"; public static final int TEZ_AM_CURATOR_CONNECTION_TIMEOUT_DEFAULT = 15000; + + @ConfigurationScope(Scope.AM) @ConfigurationProperty public static final String TEZ_FRAMEWORK_MODE = TEZ_PREFIX + ".framework.mode"; /** - * List of additional hadoop config files to load from CLASSPATH in ZOOKEEPER_STANDALONE framework mode + * List of additional hadoop config files to load from CLASSPATH in ZOOKEEPER_STANDALONE framework mode. */ @ConfigurationScope(Scope.AM) @ConfigurationProperty diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConstants.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConstants.java index 5bb50e6c67..2f2ccd5ec1 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConstants.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConstants.java @@ -42,12 +42,8 @@ public final class TezConstants { * These are the same as MR which allows Tez to run in secure * mode without configuring service ACLs */ - public static final String - TEZ_AM_SECURITY_SERVICE_AUTHORIZATION_TASK_UMBILICAL = - "security.job.task.protocol.acl"; - public static final String - TEZ_AM_SECURITY_SERVICE_AUTHORIZATION_CLIENT = - "security.job.client.protocol.acl"; + public static final String TEZ_AM_SECURITY_SERVICE_AUTHORIZATION_TASK_UMBILICAL = "security.job.task.protocol.acl"; + public static final String TEZ_AM_SECURITY_SERVICE_AUTHORIZATION_CLIENT = "security.job.client.protocol.acl"; public static final String SERVICE_PLUGINS_DESCRIPTOR_JSON = "service_plugins_descriptor.json"; public static final String TEZ_PB_BINARY_CONF_NAME = "tez-conf.pb"; diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java index f337e944db..827d333371 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java @@ -286,6 +286,7 @@ boolean createAMProxyIfNeeded() throws IOException, TezException, // attempt not running return false; } + proxy = TezClientUtils.getAMProxy(conf, appReport.getHost(), appReport.getRpcPort(), appReport.getClientToAMToken(), ugi); return true; diff --git a/tez-api/src/main/java/org/apache/tez/frameworkplugins/ClientFrameworkService.java b/tez-api/src/main/java/org/apache/tez/frameworkplugins/ClientFrameworkService.java index adbf66323d..1ee70e8344 100644 --- a/tez-api/src/main/java/org/apache/tez/frameworkplugins/ClientFrameworkService.java +++ b/tez-api/src/main/java/org/apache/tez/frameworkplugins/ClientFrameworkService.java @@ -6,9 +6,9 @@ * 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 - * + *

    + * 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. @@ -17,19 +17,26 @@ */ package org.apache.tez.frameworkplugins; -import java.util.Optional; -import org.apache.hadoop.conf.Configuration; import org.apache.tez.client.FrameworkClient; -import org.apache.tez.client.registry.AMRegistryClient; -/* - FrameworkService that runs code within the client process that is using TezClient - Bundles together a compatible FrameworkClient and AMRegistryClient +/** + * A {@code FrameworkService} that runs within the client process using {@code TezClient}. + * + *

    This service bundles together a compatible {@code FrameworkClient} and + * {@code AMRegistryClient} to enable communication and coordination with the + * Application Master.

    + * + *

    Implementations must provide a {@link FrameworkClient} instance that will + * be used by the Tez client layer.

    */ public interface ClientFrameworkService extends FrameworkService { - //Provide an impl. for org.apache.tez.client.FrameworkClient - Optional createOrGetFrameworkClient(Configuration conf); - //Provide an impl. for org.apache.tez.registry.AMRegistryClient - Optional createOrGetRegistryClient(Configuration conf); + + /** + * Create a new {@link FrameworkClient} instance used by the client-side + * Tez runtime. + * + * @return a new {@code FrameworkClient} instance + */ + FrameworkClient newFrameworkClient(); } diff --git a/tez-api/src/main/java/org/apache/tez/frameworkplugins/FrameworkMode.java b/tez-api/src/main/java/org/apache/tez/frameworkplugins/FrameworkMode.java index b7d6fe1a39..32b82b05d4 100644 --- a/tez-api/src/main/java/org/apache/tez/frameworkplugins/FrameworkMode.java +++ b/tez-api/src/main/java/org/apache/tez/frameworkplugins/FrameworkMode.java @@ -6,9 +6,9 @@ * 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 - * + *

    + * 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. @@ -27,14 +27,26 @@ A FrameworkMode is a pair of classes implementing (ClientFrameworkService, Serve public enum FrameworkMode { STANDALONE_ZOOKEEPER( - "org.apache.tez.frameworkplugins.zookeeper.ZookeeperStandaloneClientFrameworkService", - "org.apache.tez.frameworkplugins.zookeeper.ZookeeperStandaloneServerFrameworkService"); + "org.apache.tez.frameworkplugins.zookeeper.ZkStandaloneClientFrameworkService", + "org.apache.tez.frameworkplugins.zookeeper.ZkStandaloneServerFrameworkService"), - String clientClassName; - String serverClassName; + YARN( + "org.apache.tez.frameworkplugins.yarn.YarnClientFrameworkService", + "org.apache.tez.frameworkplugins.yarn.YarnServerFrameworkService"); + + private final String clientClassName; + private final String serverClassName; FrameworkMode(String clientClassName, String serverClassName) { this.clientClassName = clientClassName; this.serverClassName = serverClassName; } + + public String getClientClassName() { + return clientClassName; + } + + public String getServerClassName() { + return serverClassName; + } } diff --git a/tez-api/src/main/java/org/apache/tez/frameworkplugins/FrameworkService.java b/tez-api/src/main/java/org/apache/tez/frameworkplugins/FrameworkService.java index aff52b55a2..f27daf6577 100644 --- a/tez-api/src/main/java/org/apache/tez/frameworkplugins/FrameworkService.java +++ b/tez-api/src/main/java/org/apache/tez/frameworkplugins/FrameworkService.java @@ -6,9 +6,9 @@ * 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 - * + *

    + * 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. @@ -17,8 +17,12 @@ */ package org.apache.tez.frameworkplugins; -//Marker for a FrameworkService -//extended by ClientFrameworkService and ServerFrameworkService +/** + * Marker interface for framework-level services in Tez. + *

    + * This interface is extended by ClientFrameworkService and ServerFrameworkService + * to represent client-side and server-side framework service implementations, respectively. + *

    + */ public interface FrameworkService { - } diff --git a/tez-api/src/main/java/org/apache/tez/frameworkplugins/FrameworkUtils.java b/tez-api/src/main/java/org/apache/tez/frameworkplugins/FrameworkUtils.java index ebf6e9b3ea..51fefd6bf7 100644 --- a/tez-api/src/main/java/org/apache/tez/frameworkplugins/FrameworkUtils.java +++ b/tez-api/src/main/java/org/apache/tez/frameworkplugins/FrameworkUtils.java @@ -18,19 +18,20 @@ package org.apache.tez.frameworkplugins; import java.util.ArrayList; -import java.util.Iterator; import java.util.List; -import java.util.Optional; import java.util.ServiceLoader; import javax.annotation.Nullable; +import org.apache.hadoop.conf.Configuration; import org.apache.tez.common.ReflectionUtils; import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.dag.api.TezConstants; import org.apache.tez.dag.api.TezReflectionException; -public class FrameworkUtils { +public final class FrameworkUtils { + + private FrameworkUtils() {} /* Searches for a FrameworkService provider which implements a target interface. @@ -52,7 +53,7 @@ public class FrameworkUtils { META-INF/services/org.apache.tez.frameworkplugins.ClientFrameworkService or META-INF/services/org.apache.tez.frameworkplugins.ServerFrameworkService */ - public static Optional get(Class interfaze, @Nullable TezConfiguration conf) { + public static T get(Class interfaze, @Nullable Configuration conf) { try { if ((conf != null) && (conf.get(TezConfiguration.TEZ_FRAMEWORK_MODE) != null)) { return getByMode(interfaze, conf.get(TezConfiguration.TEZ_FRAMEWORK_MODE)); @@ -61,38 +62,35 @@ public static Optional get(Class interfaze, @ } else { return getByServiceLoader(interfaze); } - } catch(TezReflectionException e) { - throw new RuntimeException("Failed to load framework service for interface: " + interfaze.getName()); + } catch (TezReflectionException e) { + throw new RuntimeException("Failed to load framework service for interface: " + interfaze.getName(), e); } } - private static Optional getByServiceLoader(Class interfaze) { + private static T getByServiceLoader(Class interfaze) { List services = new ArrayList<>(); ServiceLoader frameworkService = ServiceLoader.load(interfaze); - Iterator it = frameworkService.iterator(); - while (it.hasNext()) { - T service = it.next(); + for (T service : frameworkService) { services.add(service); } - if(services.size() == 0) { - return Optional.empty(); + if (services.isEmpty()) { + return null; } else if (services.size() > 1) { throw new RuntimeException("Layering of multiple framework services is not supported." + " Please provide only one implementation class in configuration."); } //services is guaranteed to have one element at this point - return Optional.of(services.get(0)); + return services.getFirst(); } - private static Optional getByMode(Class interfaze, String mode) throws TezReflectionException { + private static T getByMode(Class interfaze, String mode) throws TezReflectionException { mode = mode.toUpperCase(); - String clazz = null; - if(interfaze == ClientFrameworkService.class) { - clazz = FrameworkMode.valueOf(mode).clientClassName; + String clazz; + if (interfaze == ClientFrameworkService.class) { + clazz = FrameworkMode.valueOf(mode).getClientClassName(); } else { - clazz = FrameworkMode.valueOf(mode).serverClassName; + clazz = FrameworkMode.valueOf(mode).getServerClassName(); } - return Optional.of(ReflectionUtils.createClazzInstance(clazz)); + return ReflectionUtils.createClazzInstance(clazz); } - } diff --git a/tez-api/src/main/java/org/apache/tez/frameworkplugins/package-info.java b/tez-api/src/main/java/org/apache/tez/frameworkplugins/package-info.java new file mode 100644 index 0000000000..d279462b1e --- /dev/null +++ b/tez-api/src/main/java/org/apache/tez/frameworkplugins/package-info.java @@ -0,0 +1,24 @@ +/** + * 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. + */ + +@Public +@Evolving +package org.apache.tez.frameworkplugins; + +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Evolving; diff --git a/tez-api/src/main/java/org/apache/tez/frameworkplugins/yarn/YarnClientFrameworkService.java b/tez-api/src/main/java/org/apache/tez/frameworkplugins/yarn/YarnClientFrameworkService.java new file mode 100644 index 0000000000..224f7ff256 --- /dev/null +++ b/tez-api/src/main/java/org/apache/tez/frameworkplugins/yarn/YarnClientFrameworkService.java @@ -0,0 +1,35 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.tez.frameworkplugins.yarn; + +import org.apache.hadoop.yarn.client.api.YarnClient; +import org.apache.tez.client.FrameworkClient; +import org.apache.tez.client.TezYarnClient; +import org.apache.tez.frameworkplugins.ClientFrameworkService; + +/** + * YARN-based client framework service implementation. + * Provides default YARN framework client functionality. + */ +public class YarnClientFrameworkService implements ClientFrameworkService { + + @Override + public FrameworkClient newFrameworkClient() { + return new TezYarnClient(YarnClient.createYarnClient()); + } +} diff --git a/tez-api/src/main/java/org/apache/tez/frameworkplugins/yarn/package-info.java b/tez-api/src/main/java/org/apache/tez/frameworkplugins/yarn/package-info.java new file mode 100644 index 0000000000..ffc5b78fb9 --- /dev/null +++ b/tez-api/src/main/java/org/apache/tez/frameworkplugins/yarn/package-info.java @@ -0,0 +1,24 @@ +/** + * 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. + */ + +@Public +@Evolving +package org.apache.tez.frameworkplugins.yarn; + +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Evolving; diff --git a/tez-api/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZkStandaloneClientFrameworkService.java b/tez-api/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZkStandaloneClientFrameworkService.java new file mode 100644 index 0000000000..43b11140cc --- /dev/null +++ b/tez-api/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZkStandaloneClientFrameworkService.java @@ -0,0 +1,29 @@ +/** + * 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.tez.frameworkplugins.zookeeper; + + +import org.apache.tez.client.FrameworkClient; +import org.apache.tez.client.registry.zookeeper.ZkFrameworkClient; +import org.apache.tez.frameworkplugins.ClientFrameworkService; + +public class ZkStandaloneClientFrameworkService implements ClientFrameworkService { + @Override public FrameworkClient newFrameworkClient() { + return new ZkFrameworkClient(); + } +} diff --git a/tez-api/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZookeeperStandaloneClientFrameworkService.java b/tez-api/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZookeeperStandaloneClientFrameworkService.java deleted file mode 100644 index 0d9f8c26c8..0000000000 --- a/tez-api/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZookeeperStandaloneClientFrameworkService.java +++ /dev/null @@ -1,26 +0,0 @@ -package org.apache.tez.frameworkplugins.zookeeper; - -import java.util.Optional; - -import org.apache.hadoop.conf.Configuration; -import org.apache.tez.client.FrameworkClient; -import org.apache.tez.client.registry.AMRegistryClient; -import org.apache.tez.client.registry.zookeeper.ZkAMRegistryClient; -import org.apache.tez.client.registry.zookeeper.ZkFrameworkClient; -import org.apache.tez.frameworkplugins.ClientFrameworkService; - -public class ZookeeperStandaloneClientFrameworkService implements ClientFrameworkService { - @Override public Optional createOrGetFrameworkClient(Configuration conf) { - return Optional.of(new ZkFrameworkClient()); - } - - @Override public Optional createOrGetRegistryClient(Configuration conf) { - ZkAMRegistryClient registry = ZkAMRegistryClient.getClient(conf); - try { - registry.start(); - } catch (Exception e) { - throw new RuntimeException(e); - } - return Optional.of(registry); - } -} diff --git a/tez-api/src/main/java/org/apache/tez/frameworkplugins/zookeeper/package-info.java b/tez-api/src/main/java/org/apache/tez/frameworkplugins/zookeeper/package-info.java new file mode 100644 index 0000000000..3acc21feae --- /dev/null +++ b/tez-api/src/main/java/org/apache/tez/frameworkplugins/zookeeper/package-info.java @@ -0,0 +1,24 @@ +/** + * 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. + */ + +@Public +@Evolving +package org.apache.tez.frameworkplugins.zookeeper; + +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Evolving; diff --git a/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java b/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java index 1af958514a..fb69241a4d 100644 --- a/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java +++ b/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java @@ -222,12 +222,12 @@ public void testTezClientSession() throws Exception { } @Test (timeout = 5000) - public void testTezclientReconnect() throws Exception { + public void testTezClientReconnect() throws Exception { testTezClientReconnect(true); } @Test (timeout = 5000, expected = IllegalStateException.class) - public void testTezclientReconnectNoSession() throws Exception { + public void testTezClientReconnectNoSession() throws Exception { testTezClientReconnect(false); } @@ -517,7 +517,8 @@ public void testTezClientReconnect(boolean isSession) throws Exception { TezClientForTest client2 = configureAndCreateTezClient(lrs, isSession, null); //Submission Context 2 - ArgumentCaptor captorClient2 = ArgumentCaptor.forClass(ApplicationSubmissionContext.class); + ArgumentCaptor captorClient2 = + ArgumentCaptor.forClass(ApplicationSubmissionContext.class); when(client2.mockYarnClient.getApplicationReport(client2.mockAppId).getYarnApplicationState()) .thenReturn(YarnApplicationState.RUNNING); @@ -854,6 +855,7 @@ public void testClientBuilder() { // Session mode via conf tezClient = TezClient.newBuilder("client", tezConfWitSession).build(); + assertTrue(tezClient.isSession); assertNull(tezClient.servicePluginsDescriptor); assertNotNull(tezClient.apiVersionInfo); amConf = tezClient.amConfig; @@ -865,6 +867,7 @@ public void testClientBuilder() { // Non-Session mode via conf tezClient = TezClient.newBuilder("client", tezConfNoSession).build(); + assertFalse(tezClient.isSession); assertNull(tezClient.servicePluginsDescriptor); assertNotNull(tezClient.apiVersionInfo); amConf = tezClient.amConfig; @@ -876,6 +879,7 @@ public void testClientBuilder() { // no-session via config. API explicit session. tezClient = TezClient.newBuilder("client", tezConfNoSession).setIsSession(true).build(); + assertTrue(tezClient.isSession); assertNull(tezClient.servicePluginsDescriptor); assertNotNull(tezClient.apiVersionInfo); amConf = tezClient.amConfig; @@ -889,6 +893,7 @@ public void testClientBuilder() { tezClient = TezClient.newBuilder("client", tezConfWitSession).setCredentials(credentials) .setLocalResources(localResourceMap).setServicePluginDescriptor(servicePluginsDescriptor) .build(); + assertTrue(tezClient.isSession); assertEquals(servicePluginsDescriptor, tezClient.servicePluginsDescriptor); assertNotNull(tezClient.apiVersionInfo); amConf = tezClient.amConfig; diff --git a/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java b/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java index 43acde2965..56d4192df2 100644 --- a/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java +++ b/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java @@ -703,16 +703,20 @@ public void testConfYarnZkWorkaround() { String val = "localhost:2181"; conf.set("yarn.resourcemanager.zk-address", val); - Map expected = new HashMap(); + Map expected = new HashMap<>(); expected.put("yarn.resourcemanager.zk-address", val); ConfigurationProto confProto = TezClientUtils.createFinalConfProtoForApp(conf, null); for (PlanKeyValuePair kvPair : confProto.getConfKeyValuesList()) { - String v = expected.remove(kvPair.getKey()); - assertEquals(v, kvPair.getValue()); + if (expected.containsKey(kvPair.getKey())) { // fix for polluting keys + String v = expected.remove(kvPair.getKey()); + // this way the test still validates that the original + // key/value pairs can be found in the proto's conf + assertEquals("Unexpected value for key: " + kvPair.getKey(), v, kvPair.getValue()); + } } - assertTrue(expected.isEmpty()); + assertTrue("Expected keys not found in conf: " + expected.keySet(), expected.isEmpty()); } @Test (timeout = 5000) diff --git a/tez-api/src/test/java/org/apache/tez/client/registry/zookeeper/TestZkFrameworkClient.java b/tez-api/src/test/java/org/apache/tez/client/registry/zookeeper/TestZkFrameworkClient.java new file mode 100644 index 0000000000..9a2ce0a033 --- /dev/null +++ b/tez-api/src/test/java/org/apache/tez/client/registry/zookeeper/TestZkFrameworkClient.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.tez.client.registry.zookeeper; + +import static org.junit.Assert.*; + +import java.io.File; +import java.nio.charset.StandardCharsets; + +import org.apache.commons.io.IOUtils; +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.test.TestingServer; +import org.apache.hadoop.registry.client.binding.RegistryUtils; +import org.apache.hadoop.registry.client.types.ServiceRecord; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ApplicationReport; +import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; +import org.apache.hadoop.yarn.api.records.YarnApplicationState; +import org.apache.hadoop.yarn.client.api.YarnClientApplication; +import org.apache.tez.client.registry.AMRecord; +import org.apache.tez.dag.api.TezConfiguration; +import org.apache.zookeeper.CreateMode; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Unit tests for {@link ZkFrameworkClient}. + *

    + * This test class validates the ZooKeeper-based framework client that discovers + * and communicates with Application Masters through ZooKeeper registry. + *

    + */ +public class TestZkFrameworkClient { + private static final Logger LOG = LoggerFactory.getLogger(TestZkFrameworkClient.class); + private static final File TEST_DIR = new File(System.getProperty("test.build.data", "target"), + TestZkFrameworkClient.class.getName()).getAbsoluteFile(); + + private TestingServer zkServer; + private ZkFrameworkClient zkFrameworkClient; + private CuratorFramework curatorClient; + + @Before + public void setup() throws Exception { + zkServer = new TestingServer(true); + LOG.info("Started ZooKeeper test server on port: {}", zkServer.getPort()); + } + + @After + public void teardown() throws Exception { + if (zkFrameworkClient != null) { + zkFrameworkClient.close(); + } + IOUtils.closeQuietly(curatorClient); + IOUtils.closeQuietly(zkServer); + } + + /** + * Tests initialization and lifecycle methods of ZkFrameworkClient. + */ + @Test + public void testInitAndLifecycle() throws Exception { + TezConfiguration tezConf = createTezConf(); + + zkFrameworkClient = new ZkFrameworkClient(); + zkFrameworkClient.init(tezConf); + + assertTrue("Client should be running after init", zkFrameworkClient.isRunning()); + + zkFrameworkClient.start(); + assertTrue("Client should be running after start", zkFrameworkClient.isRunning()); + + zkFrameworkClient.stop(); + assertFalse("Client should not be running after stop", zkFrameworkClient.isRunning()); + } + + /** + * Tests retrieving application report when AM is registered in ZooKeeper. + */ + @Test + public void testGetApplicationReportWithRegisteredAM() throws Exception { + TezConfiguration tezConf = createTezConf(); + + // Register a mock AM in ZooKeeper + ApplicationId appId = ApplicationId.newInstance(System.currentTimeMillis(), 1); + String testHostName = "test-host"; + String testHostIp = "127.0.0.1"; + int testPort = 12345; + registerMockAM(tezConf, appId, testHostName, testHostIp, testPort); + + zkFrameworkClient = new ZkFrameworkClient(); + zkFrameworkClient.init(tezConf); + zkFrameworkClient.start(); + + // Give time for ZK registry to initialize + Thread.sleep(500); + + ApplicationReport report = zkFrameworkClient.getApplicationReport(appId); + + assertNotNull("Application report should not be null", report); + assertEquals("Application ID should match", appId, report.getApplicationId()); + assertEquals("Host should match", testHostName, report.getHost()); + assertEquals("Port should match", testPort, report.getRpcPort()); + assertEquals("Application state should be RUNNING", YarnApplicationState.RUNNING, + report.getYarnApplicationState()); + assertEquals("AM host should be cached", testHostName, zkFrameworkClient.getAmHost()); + assertEquals("AM port should be cached", testPort, zkFrameworkClient.getAmPort()); + } + + /** + * Tests retrieving application report when AM is not found in ZooKeeper. + */ + @Test + public void testGetApplicationReportWithMissingAM() throws Exception { + TezConfiguration tezConf = createTezConf(); + + zkFrameworkClient = new ZkFrameworkClient(); + zkFrameworkClient.init(tezConf); + zkFrameworkClient.start(); + + // Give time for ZK registry to initialize + Thread.sleep(500); + + ApplicationId appId = ApplicationId.newInstance(System.currentTimeMillis(), 1); + ApplicationReport report = zkFrameworkClient.getApplicationReport(appId); + + assertNotNull("Application report should not be null", report); + assertEquals("Application ID should match", appId, report.getApplicationId()); + assertEquals("Application state should be FINISHED", YarnApplicationState.FINISHED, + report.getYarnApplicationState()); + assertEquals("Final status should be FAILED", FinalApplicationStatus.FAILED, + report.getFinalApplicationStatus()); + assertTrue("Diagnostics should mention missing AM", + report.getDiagnostics().contains("AM record not found")); + } + + /** + * Tests creating application from AM record. + */ + @Test + public void testCreateApplication() throws Exception { + TezConfiguration tezConf = createTezConf(); + + // Register a mock AM in ZooKeeper + ApplicationId appId = ApplicationId.newInstance(System.currentTimeMillis(), 1); + registerMockAM(tezConf, appId, "test-host", "127.0.0.1", 12345); + + zkFrameworkClient = new ZkFrameworkClient(); + zkFrameworkClient.init(tezConf); + zkFrameworkClient.start(); + + // Give time for ZK registry to initialize + Thread.sleep(500); + + // Need to call getApplicationReport first to populate amRecord + zkFrameworkClient.getApplicationReport(appId); + + YarnClientApplication clientApp = zkFrameworkClient.createApplication(); + + assertNotNull("YarnClientApplication should not be null", clientApp); + assertNotNull("ApplicationSubmissionContext should not be null", clientApp.getApplicationSubmissionContext()); + assertEquals("Application ID should match", appId, clientApp.getApplicationSubmissionContext().getApplicationId()); + assertNotNull("GetNewApplicationResponse should not be null", clientApp.getNewApplicationResponse()); + assertEquals("Response application ID should match", + appId, clientApp.getNewApplicationResponse().getApplicationId()); + } + + /** + * Tests kill application method. + */ + @Test + public void testKillApplication() throws Exception { + TezConfiguration tezConf = createTezConf(); + + zkFrameworkClient = new ZkFrameworkClient(); + zkFrameworkClient.init(tezConf); + zkFrameworkClient.start(); + + // Give time for ZK registry to initialize + Thread.sleep(500); + + ApplicationId appId = ApplicationId.newInstance(System.currentTimeMillis(), 1); + + // Should not throw exception + zkFrameworkClient.killApplication(appId); + } + + private TezConfiguration createTezConf() { + TezConfiguration tezConf = new TezConfiguration(); + tezConf.set(TezConfiguration.TEZ_FRAMEWORK_MODE, "STANDALONE_ZOOKEEPER"); + tezConf.set(TezConfiguration.TEZ_AM_ZOOKEEPER_QUORUM, "localhost:" + zkServer.getPort()); + tezConf.set(TezConfiguration.TEZ_AM_REGISTRY_NAMESPACE, "/tez-test-" + System.currentTimeMillis()); + tezConf.setBoolean(TezConfiguration.TEZ_LOCAL_MODE, true); + tezConf.set(TezConfiguration.TEZ_AM_STAGING_DIR, TEST_DIR.toString()); + return tezConf; + } + + private void registerMockAM(TezConfiguration tezConf, ApplicationId appId, String hostName, String hostIp, int port) throws Exception { + // Create AM record and publish it directly to ZooKeeper + AMRecord amRecord = new AMRecord(appId, hostName, hostIp, port, "test-external-id", "test-compute"); + ServiceRecord serviceRecord = amRecord.toServiceRecord(); + + RegistryUtils.ServiceRecordMarshal marshal = new RegistryUtils.ServiceRecordMarshal(); + String json = marshal.toJson(serviceRecord); + + // Use Curator to write directly to ZooKeeper + ZkConfig zkConfig = new ZkConfig(tezConf); + curatorClient = zkConfig.createCuratorFramework(); + curatorClient.start(); + + // Wait for connection + curatorClient.blockUntilConnected(); + + String namespace = zkConfig.getZkNamespace(); + String path = namespace + "/" + appId.toString(); + + // Create parent directories if needed + curatorClient.create().creatingParentsIfNeeded().withMode(CreateMode.EPHEMERAL) + .forPath(path, json.getBytes(StandardCharsets.UTF_8)); + + LOG.info("Registered mock AM to ZK path: {}", path); + } +} diff --git a/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java b/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java index 3f93de92d9..cb7581981b 100644 --- a/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java +++ b/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java @@ -94,11 +94,11 @@ public static Configuration readTezConfigurationXml(InputStream is) { } public static ConfigurationProto loadConfProtoFromText() throws IOException { - try(InputStream cis = ClassLoader.getSystemResourceAsStream(TezConfiguration.TEZ_SITE_XML); - InputStream sis = ClassLoader.getSystemResourceAsStream(TezConstants.SERVICE_PLUGINS_DESCRIPTOR_JSON)) { + try (InputStream cis = ClassLoader.getSystemResourceAsStream(TezConfiguration.TEZ_SITE_XML); + InputStream sis = ClassLoader.getSystemResourceAsStream(TezConstants.SERVICE_PLUGINS_DESCRIPTOR_JSON)) { Configuration confFromXml = TezUtilsInternal.readTezConfigurationXml(cis); - for(String confFile : confFromXml.getTrimmedStringCollection(TezConfiguration.TEZ_AM_STANDALONE_CONFS)) { - try(InputStream additionalInput = ClassLoader.getSystemResourceAsStream(confFile)) { + for (String confFile : confFromXml.getTrimmedStringCollection(TezConfiguration.TEZ_AM_STANDALONE_CONFS)) { + try (InputStream additionalInput = ClassLoader.getSystemResourceAsStream(confFile)) { Configuration additionalConfFromXml = TezUtilsInternal.readTezConfigurationXml(additionalInput); confFromXml.addResource(additionalConfFromXml); } diff --git a/tez-dag/findbugs-exclude.xml b/tez-dag/findbugs-exclude.xml index 9ee1b3d946..2c878bb6fe 100644 --- a/tez-dag/findbugs-exclude.xml +++ b/tez-dag/findbugs-exclude.xml @@ -273,4 +273,11 @@ + + + + + + + diff --git a/tez-dag/pom.xml b/tez-dag/pom.xml index 37ea5432d7..06e2340f06 100644 --- a/tez-dag/pom.xml +++ b/tez-dag/pom.xml @@ -44,10 +44,6 @@ org.apache.tez tez-api
    - - org.apache.tez - tez-mapreduce - org.apache.tez tez-common @@ -159,7 +155,6 @@ org.apache.curator curator-test - ${curator.version} test diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientServer.java b/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientServer.java index 1c62b0789a..4c5a0039e6 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientServer.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientServer.java @@ -52,7 +52,6 @@ public class DAGClientServer extends AbstractService { DAGClientHandler realInstance; Server server; InetSocketAddress bindAddress; - ApplicationAttemptId applicationAttemptId; final FileSystem stagingFs; public DAGClientServer(DAGClientHandler realInstance, @@ -60,7 +59,6 @@ public DAGClientServer(DAGClientHandler realInstance, super("DAGClientRPCServer"); this.realInstance = realInstance; this.secretManager = new ClientToAMTokenSecretManager(attemptId, null); - this.applicationAttemptId = attemptId; this.stagingFs = stagingFs; } @@ -68,8 +66,7 @@ public DAGClientServer(DAGClientHandler realInstance, public void serviceStart() { try { Configuration conf = getConfig(); - int rpcPort = conf.getInt(TezConfiguration.TEZ_AM_RPC_PORT, - TezConfiguration.TEZ_AM_RPC_PORT_DEFAULT); + int rpcPort = conf.getInt(TezConfiguration.TEZ_AM_RPC_PORT, TezConfiguration.TEZ_AM_RPC_PORT_DEFAULT); InetSocketAddress addr = new InetSocketAddress(rpcPort); DAGClientAMProtocolBlockingPBServerImpl service = @@ -86,7 +83,7 @@ public void serviceStart() { server = createServer(DAGClientAMProtocolBlockingPB.class, addr, conf, numHandlers, blockingService, TezConfiguration.TEZ_AM_CLIENT_AM_PORT_RANGE); - + // Enable service authorization? if (conf.getBoolean( CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION, @@ -120,7 +117,7 @@ public void serviceStop() { public InetSocketAddress getBindAddress() { return bindAddress; } - + public void setClientAMSecretKey(ByteBuffer key) { if (key != null && key.hasRemaining()) { // non-empty key. must be useful diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/client/registry/zookeeper/ZkAMRegistry.java b/tez-dag/src/main/java/org/apache/tez/dag/api/client/registry/zookeeper/ZkAMRegistry.java index c0daecef38..83e9c8015f 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/api/client/registry/zookeeper/ZkAMRegistry.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/api/client/registry/zookeeper/ZkAMRegistry.java @@ -6,9 +6,9 @@ * 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 - * + *

    + * 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. @@ -18,10 +18,10 @@ package org.apache.tez.dag.api.client.registry.zookeeper; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Collections; import java.util.List; -import java.util.Optional; import org.apache.curator.RetryLoop; import org.apache.curator.RetryPolicy; @@ -42,19 +42,20 @@ /** * Curator/Zookeeper impl of AMRegistry (for internal use only) - * Clients should use org.apache.tez.dag.api.client.registry.zookeeper.ZkAMRegistryClient instead + * Clients should use org.apache.tez.dag.api.client.registry.zookeeper.ZkAMRegistryClient instead. */ @InterfaceAudience.Private public class ZkAMRegistry extends AMRegistry { private static final Logger LOG = LoggerFactory.getLogger(ZkAMRegistry.class); + private final List amRecords = new ArrayList<>(); + private final String externalId; + private CuratorFramework client = null; private String namespace = null; - private List amRecords = new ArrayList<>(); private ZkConfig zkConfig = null; private boolean started = false; - private String externalId; public ZkAMRegistry(String externalId) { super("ZkAMRegistry"); @@ -63,7 +64,7 @@ public ZkAMRegistry(String externalId) { @Override public void serviceInit(Configuration conf) { - if(zkConfig == null) { + if (zkConfig == null) { zkConfig = new ZkConfig(conf); this.client = zkConfig.createCuratorFramework(); this.namespace = zkConfig.getZkNamespace(); @@ -71,8 +72,9 @@ public void serviceInit(Configuration conf) { } } - @Override public void serviceStart() throws Exception { - if(!started) { + @Override + public void serviceStart() throws Exception { + if (!started) { client.start(); started = true; LOG.info("AMRegistryZkImpl started"); @@ -80,9 +82,10 @@ public void serviceInit(Configuration conf) { } //Deletes from Zookeeper AMRecords that were added by this instance - @Override public void serviceStop() throws Exception { + @Override + public void serviceStop() throws Exception { List records = new ArrayList<>(amRecords); - for(AMRecord amRecord : records) { + for (AMRecord amRecord : records) { remove(amRecord); } client.close(); @@ -91,20 +94,23 @@ public void serviceInit(Configuration conf) { //Serialize AMRecord to ServiceRecord and deliver the JSON bytes to //zkNode at the path: / - @Override public void add(AMRecord server) throws Exception { + @Override + public void add(AMRecord server) throws Exception { RegistryUtils.ServiceRecordMarshal marshal = new RegistryUtils.ServiceRecordMarshal(); String json = marshal.toJson(server.toServiceRecord()); try { final String path = namespace + "/" + server.getApplicationId().toString(); - client.setData().forPath(path, json.getBytes()); + client.setData().forPath(path, json.getBytes(StandardCharsets.UTF_8)); LOG.info("Added AMRecord to zkpath {}", path); - } catch(KeeperException.NoNodeException nne) { - client.create().creatingParentContainersIfNeeded().withMode(CreateMode.EPHEMERAL).forPath(namespace + "/" + server.getApplicationId().toString(), json.getBytes()); + } catch (KeeperException.NoNodeException nne) { + client.create().creatingParentContainersIfNeeded().withMode(CreateMode.EPHEMERAL) + .forPath(namespace + "/" + server.getApplicationId().toString(), json.getBytes(StandardCharsets.UTF_8)); } amRecords.add(server); } - @Override public void remove(AMRecord server) throws Exception { + @Override + public void remove(AMRecord server) throws Exception { amRecords.remove(server); final String path = namespace + "/" + server.getApplicationId().toString(); client.delete().forPath(path); @@ -112,7 +118,7 @@ public void serviceInit(Configuration conf) { } @Override - public Optional generateNewId() throws Exception { + public ApplicationId generateNewId() throws Exception { createNamespaceIfNotExists(); long namespaceCreationTime = getNamespaceCreationTime(); @@ -120,11 +126,11 @@ public Optional generateNewId() throws Exception { long startTime = System.currentTimeMillis(); RetryPolicy retryPolicy = zkConfig.getRetryPolicy(); int tryId = 0; - for(int i = 0; (i < zkConfig.getCuratorMaxRetries()) && !success; i++) { + for (int i = 0; (i < zkConfig.getCuratorMaxRetries()) && !success; i++) { List children = client.getChildren().forPath(namespace); - if((children != null) && (children.size() != 0)) { - Collections.sort(children, Collections.reverseOrder()); - String last = children.get(0); + if (children != null && !children.isEmpty()) { + children.sort(Collections.reverseOrder()); + String last = children.getFirst(); ApplicationId lastAppId = ApplicationId.fromString(last); tryId = lastAppId.getId() + 1; } @@ -135,22 +141,23 @@ public Optional generateNewId() throws Exception { .withMode(CreateMode.EPHEMERAL) .forPath(namespace + "/" + tryAppId.toString(), new byte[0]); success = true; - } catch(KeeperException.NodeExistsException nodeExists) { + } catch (KeeperException.NodeExistsException nodeExists) { long elapsedTime = System.currentTimeMillis() - startTime; retryPolicy.allowRetry(i + 1, elapsedTime, RetryLoop.getDefaultRetrySleeper()); tryId++; } } - if(success) { - return Optional.of(ApplicationId.newInstance(namespaceCreationTime, tryId)); + if (success) { + return ApplicationId.newInstance(namespaceCreationTime, tryId); } else { throw new RuntimeException("Could not obtain unique ApplicationId after " + zkConfig.getCuratorMaxRetries() + " tries"); } } - @Override public AMRecord createAmRecord(ApplicationId appId, String hostName, int port) { - return new AMRecord(appId, hostName, port, externalId); + @Override + public AMRecord createAmRecord(ApplicationId appId, String hostName, String hostIp, int port, String computeName) { + return new AMRecord(appId, hostName, hostIp, port, externalId, computeName); } private long getNamespaceCreationTime() throws Exception { @@ -161,7 +168,7 @@ private long getNamespaceCreationTime() throws Exception { private void createNamespaceIfNotExists() throws Exception { try { client.create().creatingParentContainersIfNeeded().forPath(namespace); - } catch(KeeperException.NodeExistsException nodeExists) { + } catch (KeeperException.NodeExistsException nodeExists) { LOG.info("Namespace already exists, will use existing: {}", namespace); } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/client/registry/zookeeper/package-info.java b/tez-dag/src/main/java/org/apache/tez/dag/api/client/registry/zookeeper/package-info.java new file mode 100644 index 0000000000..be8343b7c5 --- /dev/null +++ b/tez-dag/src/main/java/org/apache/tez/dag/api/client/registry/zookeeper/package-info.java @@ -0,0 +1,24 @@ +/** + * 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. + */ + +@Public +@Evolving +package org.apache.tez.dag.api.client.registry.zookeeper; + +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Evolving; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java b/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java index 9afd1fb873..c3258b426a 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java @@ -38,6 +38,7 @@ import org.apache.tez.dag.app.rm.node.AMNodeTracker; import org.apache.tez.dag.history.HistoryEventHandler; import org.apache.tez.dag.records.TezDAGID; +import org.apache.tez.frameworkplugins.AMExtensions; import org.apache.tez.hadoop.shim.HadoopShim; import com.google.common.util.concurrent.ListeningExecutorService; @@ -140,4 +141,6 @@ public interface AppContext { public HadoopShim getHadoopShim(); public DAGRecoveryData getDAGRecoveryData(); + + AMExtensions getAmExtensions(); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java index 1be36212c0..58d7ba928c 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java @@ -20,13 +20,10 @@ -import java.io.BufferedReader; -import java.io.ByteArrayInputStream; import java.io.File; import java.io.FileInputStream; import java.io.FileNotFoundException; import java.io.IOException; -import java.io.InputStreamReader; import java.io.PrintWriter; import java.net.URI; import java.net.URISyntaxException; @@ -48,7 +45,6 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Objects; -import java.util.Optional; import java.util.Random; import java.util.Set; import java.util.Timer; @@ -92,7 +88,6 @@ import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.security.AMRMTokenIdentifier; import org.apache.hadoop.yarn.util.Clock; -import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree; import org.apache.hadoop.yarn.util.SystemClock; import org.apache.log4j.helpers.ThreadLocalMap; @@ -101,6 +96,7 @@ import org.apache.tez.client.TezClientUtils; import org.apache.tez.client.registry.AMRecord; import org.apache.tez.client.registry.AMRegistry; +import org.apache.tez.client.registry.zookeeper.ZkConfig; import org.apache.tez.common.AsyncDispatcher; import org.apache.tez.common.AsyncDispatcherConcurrent; import org.apache.tez.common.ContainerSignatureMatcher; @@ -186,9 +182,10 @@ import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.dag.utils.RelocalizationUtils; import org.apache.tez.dag.utils.Simple2LevelVersionComparator; -import org.apache.tez.frameworkplugins.AmExtensions; +import org.apache.tez.frameworkplugins.AMExtensions; import org.apache.tez.frameworkplugins.FrameworkUtils; import org.apache.tez.frameworkplugins.ServerFrameworkService; +import org.apache.tez.frameworkplugins.yarn.YarnServerFrameworkService; import org.apache.tez.hadoop.shim.HadoopShim; import org.apache.tez.hadoop.shim.HadoopShimsLoader; import org.apache.tez.runtime.hook.TezDAGHook; @@ -228,10 +225,7 @@ public class DAGAppMaster extends AbstractService { private static final Logger LOG = LoggerFactory.getLogger(DAGAppMaster.class); - private static Optional frameworkService = - FrameworkUtils.get(ServerFrameworkService.class, null); - public static Optional amExts = - frameworkService.flatMap(fs -> fs.createOrGetDAGAppMasterExtensions()); + private ServerFrameworkService frameworkService; /** * Priority of the DAGAppMaster shutdown hook. @@ -433,6 +427,8 @@ private long getAMGCTime() { protected void serviceInit(final Configuration conf) throws Exception { this.amConf = conf; + this.frameworkService = getFrameworkService(conf); + initResourceCalculatorPlugins(); this.hadoopShim = new HadoopShimsLoader(this.amConf).getHadoopShim(); @@ -495,23 +491,20 @@ protected void serviceInit(final Configuration conf) throws Exception { addIfService(dispatcher, false); - - recoveryDataDir = TezCommonUtils.getRecoveryPath(tezSystemStagingDir, conf); - recoveryFS = recoveryDataDir.getFileSystem(conf); - currentRecoveryDataDir = TezCommonUtils.getAttemptRecoveryPath(recoveryDataDir, - appAttemptID.getAttemptId()); - if (LOG.isDebugEnabled()) { - LOG.debug("Stage directory information for AppAttemptId :" + this.appAttemptID - + " tezSystemStagingDir :" + tezSystemStagingDir + " recoveryDataDir :" + recoveryDataDir - + " recoveryAttemptDir :" + currentRecoveryDataDir); - } + recoveryDataDir = TezCommonUtils.getRecoveryPath(tezSystemStagingDir, conf); + recoveryFS = recoveryDataDir.getFileSystem(conf); + currentRecoveryDataDir = TezCommonUtils.getAttemptRecoveryPath(recoveryDataDir, + appAttemptID.getAttemptId()); + if (LOG.isDebugEnabled()) { + LOG.debug("Stage directory information for AppAttemptId :" + this.appAttemptID + + " tezSystemStagingDir :" + tezSystemStagingDir + " recoveryDataDir :" + recoveryDataDir + + " recoveryAttemptDir :" + currentRecoveryDataDir); + } recoveryEnabled = conf.getBoolean(TezConfiguration.DAG_RECOVERY_ENABLED, TezConfiguration.DAG_RECOVERY_ENABLED_DEFAULT); initClientRpcServer(); - - taskHeartbeatHandler = createTaskHeartbeatHandler(context, conf); addIfService(taskHeartbeatHandler, true); @@ -520,12 +513,14 @@ protected void serviceInit(final Configuration conf) throws Exception { jobTokenSecretManager = new JobTokenSecretManager(amConf); - sessionToken = amExts.flatMap(amExtensions -> amExtensions.getSessionToken( - appAttemptID, jobTokenSecretManager, amCredentials - )).orElse(TokenCache.getSessionToken(amCredentials)); + sessionToken = frameworkService.getAMExtensions().getSessionToken( + appAttemptID, jobTokenSecretManager, amCredentials); if (sessionToken == null) { - throw new RuntimeException("Could not find session token in AM Credentials"); + sessionToken = TokenCache.getSessionToken(amCredentials); + if (sessionToken == null) { + throw new RuntimeException("Could not find session token in AM Credentials"); + } } // Prepare the TaskAttemptListener server for authentication of Containers @@ -609,12 +604,8 @@ protected void serviceInit(final Configuration conf) throws Exception { if (!versionMismatch) { if (isSession) { - DAGProtos.PlanLocalResourcesProto amLocalResourceProto = - amExts.flatMap(amExtensions -> amExtensions.getAdditionalSessionResources(workingDirectory)) - .orElse(null); - if(amLocalResourceProto == null) { - amLocalResourceProto = getAdditionalSessionResources(workingDirectory); - } + PlanLocalResourcesProto amLocalResourceProto = + frameworkService.getAMExtensions().getAdditionalSessionResources(workingDirectory); amResources.putAll(DagTypeConverters.convertFromPlanLocalResources(amLocalResourceProto)); } } @@ -622,16 +613,15 @@ protected void serviceInit(final Configuration conf) throws Exception { int threadCount = conf.getInt(TezConfiguration.TEZ_AM_DAG_APPCONTEXT_THREAD_COUNT_LIMIT, TezConfiguration.TEZ_AM_DAG_APPCONTEXT_THREAD_COUNT_LIMIT_DEFAULT); // NOTE: LinkedBlockingQueue does not have a capacity Limit and can thus - // occupy large memory chunks when numerous Runables are pending for execution + // occupy large memory chunks when numerous Runnables are pending for execution ExecutorService rawExecutor = Executors.newFixedThreadPool(threadCount, new ThreadFactoryBuilder() .setDaemon(true).setNameFormat("App Shared Pool - #%d").build()); execService = MoreExecutors.listeningDecorator(rawExecutor); - Optional amRegistry = - frameworkService.flatMap(service -> service.createOrGetAMRegistry(conf)); - if(amRegistry.isPresent()) { - initAmRegistry(appAttemptID.getApplicationId(), amRegistry.get(), clientRpcServer); - addIfService(amRegistry.get(), false); + AMRegistry amRegistry = frameworkService.getAMRegistry(conf); + if (amRegistry != null) { + initAmRegistry(appAttemptID.getApplicationId(), amRegistry, clientRpcServer); + addIfService(amRegistry, false); } initServices(conf); @@ -656,25 +646,20 @@ protected void serviceInit(final Configuration conf) throws Exception { } } - private static DAGProtos.PlanLocalResourcesProto getAdditionalSessionResources(String workingDirectory) throws IOException { - FileInputStream sessionResourcesStream = null; - try { - sessionResourcesStream = - new FileInputStream(new File(workingDirectory, TezConstants.TEZ_AM_LOCAL_RESOURCES_PB_FILE_NAME)); - return DAGProtos.PlanLocalResourcesProto.parseDelimitedFrom(sessionResourcesStream); - } finally { - if (sessionResourcesStream != null) { - sessionResourcesStream.close(); - } - } + private static ServerFrameworkService getFrameworkService(Configuration conf) { + ServerFrameworkService frameworkService = FrameworkUtils.get(ServerFrameworkService.class, conf); + return frameworkService == null ? new YarnServerFrameworkService() : frameworkService; } @VisibleForTesting - public static void initAmRegistry(ApplicationId appId, AMRegistry amRegistry, DAGClientServer dagClientServer) throws Exception { + public void initAmRegistry(ApplicationId appId, AMRegistry amRegistry, DAGClientServer dagClientServer) { dagClientServer.registerServiceListener((service) -> { if (service.isInState(STATE.STARTED)) { + final String computeName = System.getenv(ZkConfig.COMPUTE_GROUP_NAME_ENV); AMRecord amRecord = amRegistry.createAmRecord( - appId, dagClientServer.getBindAddress().getHostName(), dagClientServer.getBindAddress().getPort() + appId, dagClientServer.getBindAddress().getHostName(), + dagClientServer.getBindAddress().getAddress().getHostAddress(), + dagClientServer.getBindAddress().getPort(), computeName ); try { amRegistry.add(amRecord); @@ -1791,6 +1776,11 @@ public String getQueueName() { public void setQueueName(String queueName) { this.queueName = queueName; } + + @Override + public AMExtensions getAmExtensions() { + return frameworkService.getAMExtensions(); + } } private String getShutdownTimeString() { @@ -2052,7 +2042,7 @@ public void serviceStart() throws Exception { this.lastDAGCompletionTime = clock.getTime(); DAGRecoveryData recoveredDAGData = null; - if(recoveryEnabled) { + if (recoveryEnabled) { try { recoveredDAGData = recoverDAG(); } catch (IOException e) { @@ -2408,8 +2398,6 @@ public static void main(String[] args) { Thread.setDefaultUncaughtExceptionHandler(new YarnUncaughtExceptionHandler()); final String pid = System.getenv().get("JVM_PID"); - String containerIdStr = System.getenv(ApplicationConstants.Environment.CONTAINER_ID.name()); - String nodeHostString = System.getenv(ApplicationConstants.Environment.NM_HOST.name()); String nodePortString = System.getenv(ApplicationConstants.Environment.NM_PORT.name()); String nodeHttpPortString = System.getenv(ApplicationConstants.Environment.NM_HTTP_PORT.name()); @@ -2424,17 +2412,12 @@ public static void main(String[] args) { Configuration conf = new Configuration(); - DAGProtos.ConfigurationProto confProto = amExts.flatMap(amExt -> amExt.loadConfigurationProto()).orElse(null); - if (confProto == null) { - confProto = TezUtilsInternal - .readUserSpecifiedTezConfiguration(System.getenv(ApplicationConstants.Environment.PWD.name())); - } + AMExtensions amExtensions = getFrameworkService(conf).getAMExtensions(); + DAGProtos.ConfigurationProto confProto = amExtensions.loadConfigurationProto(); TezUtilsInternal.addUserSpecifiedTezConfiguration(conf, confProto.getConfKeyValuesList()); - ContainerId containerId = amExts.flatMap(amExt -> amExt.allocateContainerId(conf)).orElse(null); - if (containerId == null) { - containerId = ConverterUtils.toContainerId(containerIdStr); - } + ContainerId containerId = amExtensions.allocateContainerId(conf); + ApplicationAttemptId applicationAttemptId = containerId.getApplicationAttemptId(); org.apache.hadoop.ipc.CallerContext.setCurrent(new org.apache.hadoop.ipc.CallerContext .Builder("tez_appmaster_" + containerId.getApplicationAttemptId() @@ -2538,6 +2521,7 @@ public void run() { } else if (appMaster.state == DAGAppMasterState.RUNNING) { appMaster.state = DAGAppMasterState.ERROR; } + appMaster.stop(); } @@ -2743,20 +2727,4 @@ public void vertexComplete(TezVertexID completedVertexID, Set nodesList) public void taskAttemptFailed(TezTaskAttemptID attemptID, NodeId nodeId) { getContainerLauncherManager().taskAttemptFailed(attemptID, jobTokenSecretManager, nodeId); } - - private static DAGProtos.ConfigurationProto getConfigurationProtoFromText() throws IOException { - Configuration configuration = new Configuration(); - String baseDir = System.getenv("TEZ_CONF_DIR"); - BufferedReader br = new BufferedReader( - new InputStreamReader(new FileInputStream(baseDir + '/' + "tez-site.xml"))); - StringBuilder builder = new StringBuilder(); - String line = br.readLine(); - while (line != null) { - builder.append(line); - line = br.readLine(); - } - byte[] bytes = builder.toString().getBytes(); - configuration.addResource(new ByteArrayInputStream(bytes)); - return TezClientUtils.createFinalConfProtoForApp(configuration, null); - } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/PluginManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/PluginManager.java index b9151bd4cc..1b1307deb3 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/PluginManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/PluginManager.java @@ -174,6 +174,11 @@ public static void parsePlugin(List resultList, for (TezNamedEntityDescriptorProto namedEntityDescriptorProto : namedEntityDescriptorProtos) { NamedEntityDescriptor descriptor = DagTypeConverters .convertNamedDescriptorFromProto(namedEntityDescriptorProto); + if (descriptor.getUserPayload() == null) { + //If custom-plugin descriptor includes no payload, include the defaultPayload + //Useful in providing Configuration payload for hand-written JSON descriptors + descriptor.setUserPayload(defaultPayload); + } addDescriptor(resultList, pluginMap, descriptor); } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java b/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java index 7e2452525d..04812dff17 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java @@ -46,7 +46,6 @@ import org.apache.tez.dag.app.dag.impl.DAGImpl; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; -import org.apache.tez.dag.history.events.*; import org.apache.tez.dag.history.events.AMLaunchedEvent; import org.apache.tez.dag.history.events.AMStartedEvent; import org.apache.tez.dag.history.events.ContainerLaunchedEvent; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java index 7794a07047..b2e750bb21 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java @@ -159,8 +159,7 @@ protected void startRpcServer() { conf.getInt(TezConfiguration.TEZ_AM_TASK_LISTENER_THREAD_COUNT, TezConfiguration.TEZ_AM_TASK_LISTENER_THREAD_COUNT_DEFAULT)) .setPortRangeConfig(TezConfiguration.TEZ_AM_TASK_AM_PORT_RANGE) - .setSecretManager(jobTokenSecretManager) - .build(); + .setSecretManager(jobTokenSecretManager).build(); // Enable service authorization? if (conf.getBoolean( diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java index 5f08e33afe..941ea8fbd2 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java @@ -92,9 +92,8 @@ import org.apache.tez.dag.api.records.DAGProtos.PlanVertexGroupInfo; import org.apache.tez.dag.api.records.DAGProtos.VertexPlan; import org.apache.tez.dag.app.AppContext; -import org.apache.tez.dag.app.DAGAppMaster; -import org.apache.tez.dag.app.RecoveryParser; import org.apache.tez.dag.app.RecoveryParser.DAGRecoveryData; +import org.apache.tez.dag.app.RecoveryParser.VertexRecoveryData; import org.apache.tez.dag.app.TaskCommunicatorManagerInterface; import org.apache.tez.dag.app.TaskHeartbeatHandler; import org.apache.tez.dag.app.dag.DAG; @@ -139,6 +138,7 @@ import org.apache.tez.dag.utils.RelocalizationUtils; import org.apache.tez.dag.utils.TaskSpecificLaunchCmdOption; import org.apache.tez.dag.utils.TezBuilderUtils; +import org.apache.tez.frameworkplugins.TaskResourceException; import org.apache.tez.runtime.api.OutputCommitter; import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; import org.apache.tez.state.OnStateChangedCallback; @@ -1291,7 +1291,7 @@ void logJobHistoryInitedEvent() { clock.getTime(), this.userName, this.dagName, this.getVertexNameIDMapping()); this.appContext.getHistoryHandler().handle( new DAGHistoryEvent(dagId, initEvt)); - } + } } void logJobHistoryStartedEvent() { @@ -1301,7 +1301,7 @@ void logJobHistoryStartedEvent() { this.startTime, this.userName, this.dagName); this.appContext.getHistoryHandler().handle( new DAGHistoryEvent(dagId, startEvt)); - } + } } void logJobHistoryFinishedEvent(TezCounters counters) throws IOException { @@ -1320,7 +1320,7 @@ void logJobHistoryFinishedEvent(TezCounters counters) throws IOException { this.jobPlan); this.appContext.getHistoryHandler().handleCriticalEvent( new DAGHistoryEvent(dagId, finishEvt)); - } + } } void logJobHistoryUnsuccesfulEvent(DAGState state, TezCounters counters) throws IOException { @@ -1340,7 +1340,7 @@ void logJobHistoryUnsuccesfulEvent(DAGState state, TezCounters counters) throws this.appContext.getApplicationAttemptId(), this.jobPlan); this.appContext.getHistoryHandler().handleCriticalEvent( new DAGHistoryEvent(dagId, finishEvt)); - } + } } // triggered by vertex_complete @@ -1613,23 +1613,14 @@ DAGState initializeDAG() { addVertex(v); } - // check task resources, only check it in non-local mode - boolean checkTaskResources = true; - if(DAGAppMaster.amExts.isPresent()) { - checkTaskResources = DAGAppMaster.amExts.get().checkTaskResources(vertexMap, appContext); - } - if (!appContext.isLocal() && checkTaskResources) { - for (Vertex v : vertexMap.values()) { - // TODO TEZ-2003 (post) TEZ-2624 Ideally, this should be per source. - if (v.getTaskResource().compareTo(appContext.getClusterInfo().getMaxContainerCapability()) > 0) { - String msg = "Vertex's TaskResource is beyond the cluster container capability," + - "Vertex=" + v.getLogIdentifier() +", Requested TaskResource=" + v.getTaskResource() - + ", Cluster MaxContainerCapability=" + appContext.getClusterInfo().getMaxContainerCapability(); - LOG.error(msg); - addDiagnostic(msg); - finished(DAGState.FAILED); - return DAGState.FAILED; - } + if (!appContext.isLocal()) { + try { + appContext.getAmExtensions().checkTaskResources(vertexMap, appContext.getClusterInfo()); + } catch (TaskResourceException e) { + LOG.error(e.getMessage()); + addDiagnostic(e.getMessage()); + finished(DAGState.FAILED); + return DAGState.FAILED; } } @@ -1774,22 +1765,22 @@ private static void parseVertexEdges(DAGImpl dag, Map edgePlan Map outVertices = new HashMap(); - for(String inEdgeId : vertexPlan.getInEdgeIdList()){ - EdgePlan edgePlan = edgePlans.get(inEdgeId); - Vertex inVertex = dag.vertexMap.get(edgePlan.getInputVertexName()); - Edge edge = dag.edges.get(inEdgeId); - edge.setSourceVertex(inVertex); - edge.setDestinationVertex(vertex); - inVertices.put(inVertex, edge); + for (String inEdgeId : vertexPlan.getInEdgeIdList()) { + EdgePlan edgePlan = edgePlans.get(inEdgeId); + Vertex inVertex = dag.vertexMap.get(edgePlan.getInputVertexName()); + Edge edge = dag.edges.get(inEdgeId); + edge.setSourceVertex(inVertex); + edge.setDestinationVertex(vertex); + inVertices.put(inVertex, edge); } - for(String outEdgeId : vertexPlan.getOutEdgeIdList()){ - EdgePlan edgePlan = edgePlans.get(outEdgeId); - Vertex outVertex = dag.vertexMap.get(edgePlan.getOutputVertexName()); - Edge edge = dag.edges.get(outEdgeId); - edge.setSourceVertex(vertex); - edge.setDestinationVertex(outVertex); - outVertices.put(outVertex, edge); + for (String outEdgeId : vertexPlan.getOutEdgeIdList()) { + EdgePlan edgePlan = edgePlans.get(outEdgeId); + Vertex outVertex = dag.vertexMap.get(edgePlan.getOutputVertexName()); + Edge edge = dag.edges.get(outEdgeId); + edge.setSourceVertex(vertex); + edge.setDestinationVertex(outVertex); + outVertices.put(outVertex, edge); } vertex.setInputVertices(inVertices); @@ -1866,6 +1857,7 @@ public DAGState transition(DAGImpl dag, DAGEvent dagEvent) { return DAGState.NEW; } } + private static class InitTransition implements MultipleArcTransition { @@ -1884,7 +1876,7 @@ public DAGState transition(DAGImpl dag, DAGEvent event) { if (dag.recoveryData != null && dag.recoveryData.getDAGInitializedEvent() != null) { dag.initTime = dag.recoveryData.getDAGInitializedEvent().getInitTime(); } else { - dag.initTime = dag.clock.getTime(); + dag.initTime = dag.clock.getTime(); } dag.startDAGCpuTime = dag.appContext.getCumulativeCPUTime(); dag.startDAGGCTime = dag.appContext.getCumulativeGCTime(); @@ -1916,7 +1908,7 @@ public void transition(DAGImpl dag, DAGEvent event) { if (dag.recoveryData != null && dag.recoveryData.getDAGStartedEvent() != null) { dag.startTime = dag.recoveryData.getDAGStartedEvent().getStartTime(); } else { - dag.startTime = dag.clock.getTime(); + dag.startTime = dag.clock.getTime(); } DAGEventStartDag startEvent = (DAGEventStartDag) event; List additionalUrlsForClasspath = startEvent.getAdditionalUrlsForClasspath(); @@ -2178,7 +2170,7 @@ private boolean vertexSucceeded(Vertex vertex) { LOG.info("VertexGroup was already committed as per recovery" + " data, groupName=" + groupInfo.groupName); for (String vertexName : groupInfo.groupMembers) { - RecoveryParser.VertexRecoveryData vertexRecoveryData = + VertexRecoveryData vertexRecoveryData = recoveryData.getVertexRecoveryData(getVertex(vertexName).getVertexId()); Preconditions.checkArgument(vertexRecoveryData != null,"Vertex Group has been committed" + ", but no VertexRecoveryData found for its vertex " + vertexName); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java index 8d69d86a4d..def53ef1c0 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java @@ -1297,6 +1297,7 @@ public TaskAttemptStateInternal transition(TaskAttemptImpl ta, TaskAttemptEvent } return TaskAttemptStateInternal.NEW; } + TaskAttemptEventSchedule scheduleEvent = (TaskAttemptEventSchedule) event; ta.scheduledTime = ta.clock.getTime(); @@ -1400,9 +1401,9 @@ public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) { if (ta.recoveryData == null || ta.recoveryData.getTaskAttemptFinishedEvent() == null) { - ta.setFinishTime(); - ta.logJobHistoryAttemptUnsuccesfulCompletion(helper - .getTaskAttemptState(), helper.getFailureType(event)); + ta.setFinishTime(); + ta.logJobHistoryAttemptUnsuccesfulCompletion(helper + .getTaskAttemptState(), helper.getFailureType(event)); } else { ta.finishTime = ta.recoveryData.getTaskAttemptFinishedEvent().getFinishTime(); ta.isRecoveredDuration = true; @@ -1635,7 +1636,6 @@ public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) { // If TaskAttempt is recovered to SUCCEEDED, send events generated by this TaskAttempt to vertex // for its downstream consumers. For normal dag execution, the events are sent by TaskAttemptListener // for performance consideration. - if (ta.recoveryData != null && ta.recoveryData.isTaskAttemptSucceeded()) { TaskAttemptFinishedEvent taFinishedEvent = ta.recoveryData .getTaskAttemptFinishedEvent(); @@ -1648,9 +1648,9 @@ public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) { ta.finishTime = taFinishedEvent.getFinishTime(); ta.isRecoveredDuration = true; } else { - ta.setFinishTime(); - // Send out history event. - ta.logJobHistoryAttemptFinishedEvent(TaskAttemptStateInternal.SUCCEEDED); + ta.setFinishTime(); + // Send out history event. + ta.logJobHistoryAttemptFinishedEvent(TaskAttemptStateInternal.SUCCEEDED); } ta.sendEvent(createDAGCounterUpdateEventTAFinished(ta, diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java index 6a78ea86eb..ffb41e364a 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java @@ -55,7 +55,7 @@ import org.apache.tez.dag.api.oldrecords.TaskState; import org.apache.tez.dag.app.AppContext; import org.apache.tez.dag.app.ContainerContext; -import org.apache.tez.dag.app.RecoveryParser; +import org.apache.tez.dag.app.RecoveryParser.TaskRecoveryData; import org.apache.tez.dag.app.TaskCommunicatorManagerInterface; import org.apache.tez.dag.app.TaskHeartbeatHandler; import org.apache.tez.dag.app.dag.StateChangeNotifier; @@ -143,7 +143,8 @@ public class TaskImpl implements Task, EventHandler { long scheduledTime; final StateChangeNotifier stateChangeNotifier; - private final RecoveryParser.TaskRecoveryData recoveryData; + private final TaskRecoveryData recoveryData; + private final List tezEventsForTaskAttempts = new ArrayList(); static final ArrayList EMPTY_TASK_ATTEMPT_TEZ_EVENTS = new ArrayList(0); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezLocalCacheManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezLocalCacheManager.java index a0268f66e1..2d044a5689 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezLocalCacheManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezLocalCacheManager.java @@ -80,8 +80,9 @@ public void localize() throws IOException { try { // construct new threads with helpful names - ThreadFactory threadFactory = - new ThreadFactoryBuilder().setNameFormat("TezLocalCacheManager Downloader #%d").build(); + ThreadFactory threadFactory = new ThreadFactoryBuilder() + .setNameFormat("TezLocalCacheManager Downloader #%d") + .build(); threadPool = Executors.newCachedThreadPool(threadFactory); // start all fetches @@ -107,7 +108,6 @@ public void localize() throws IOException { Path dest = new Path(downloadDir.toAbsolutePath().toString()); FSDownload downloader = new FSDownload(fileContext, ugi, conf, dest, resource); Future downloadedPath = threadPool.submit(downloader); - resourceInfo.put(resource, new ResourceInfo(downloadedPath, linkPath)); } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java index 19af76aa6e..c9c6178732 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java @@ -644,7 +644,7 @@ protected void instantiateSchedulers(String host, int port, String trackingUrl, // Use the provided appId instead of constructing one for containers. customAppIdIdentifier = appContext.getApplicationID().getClusterTimestamp(); } else { - customAppIdIdentifier = SCHEDULER_APP_ID_BASE + (j++ * SCHEDULER_APP_ID_INCREMENT); + customAppIdIdentifier = SCHEDULER_APP_ID_BASE + (j++ * SCHEDULER_APP_ID_INCREMENT); } taskSchedulers[i] = new TaskSchedulerWrapper(createTaskScheduler(host, port, trackingUrl, appContext, taskSchedulerDescriptors[i], customAppIdIdentifier, i)); diff --git a/tez-dag/src/main/java/org/apache/tez/frameworkplugins/AMExtensions.java b/tez-dag/src/main/java/org/apache/tez/frameworkplugins/AMExtensions.java new file mode 100644 index 0000000000..6c07960ba5 --- /dev/null +++ b/tez-dag/src/main/java/org/apache/tez/frameworkplugins/AMExtensions.java @@ -0,0 +1,92 @@ +/** + * 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.tez.frameworkplugins; + +import java.io.IOException; +import java.util.Map; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.Credentials; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.tez.common.security.JobTokenIdentifier; +import org.apache.tez.common.security.JobTokenSecretManager; +import org.apache.tez.dag.api.records.DAGProtos; +import org.apache.tez.dag.app.ClusterInfo; +import org.apache.tez.dag.app.dag.Vertex; + +/** + * Extension points for customizing AM behavior. + * + *

    These hooks allow injecting alternate or additional logic into the + * Application Master without requiring a standalone service. They are + * intended for behaviors that are too small or cross-cutting to justify + * a dedicated service.

    + */ +public interface AMExtensions { + + /** + * Override the default configuration loading performed in + * {@code DAGAppMaster.main(...)}. + * + * @return a {@link DAGProtos.ConfigurationProto} representing the final configuration + * @throws IOException if configuration loading fails + */ + DAGProtos.ConfigurationProto loadConfigurationProto() throws IOException; + + /** + * Override the default logic used to assign a {@link ContainerId} to the AM. + * + * @param conf the Tez configuration + * @return the allocated {@link ContainerId} + */ + ContainerId allocateContainerId(Configuration conf); + + /** + * Validate resource constraints for tasks before execution. + * + * @param vertices mapping of vertex names to their DAG vertices + * @param clusterInfo cluster resource information + * @throws TaskResourceException if resource requirements cannot be satisfied + */ + void checkTaskResources(Map vertices, ClusterInfo clusterInfo) throws TaskResourceException; + + /** + * Create or override the session token used for AM authentication. + * + * @param appAttemptID current application attempt ID + * @param jobTokenSecretManager token secret manager + * @param amCredentials AM credentials store + * @return the session token + */ + Token getSessionToken( + ApplicationAttemptId appAttemptID, + JobTokenSecretManager jobTokenSecretManager, + Credentials amCredentials + ); + + /** + * Provide additional local resources required for the AM session. + * + * @param workingDirectory the AM working directory + * @return protocol buffers describing local session resources + * @throws IOException if resources cannot be discovered or packaged + */ + DAGProtos.PlanLocalResourcesProto getAdditionalSessionResources(String workingDirectory) throws IOException; +} diff --git a/tez-dag/src/main/java/org/apache/tez/frameworkplugins/AmExtensions.java b/tez-dag/src/main/java/org/apache/tez/frameworkplugins/AmExtensions.java deleted file mode 100644 index 5782944281..0000000000 --- a/tez-dag/src/main/java/org/apache/tez/frameworkplugins/AmExtensions.java +++ /dev/null @@ -1,63 +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.tez.frameworkplugins; - -import java.util.Map; -import java.util.Optional; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.security.Credentials; -import org.apache.hadoop.security.token.Token; -import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.tez.common.security.JobTokenIdentifier; -import org.apache.tez.common.security.JobTokenSecretManager; -import org.apache.tez.dag.api.records.DAGProtos; -import org.apache.tez.dag.app.AppContext; -import org.apache.tez.dag.app.dag.Vertex; - -/* - Plugin points to provide alternate AM behavior that - is either too small or too scattered to be usefully encapsulated as its own service - */ -public interface AmExtensions { - - //Override default Configuration loading at DAGAppMaster.main - default Optional loadConfigurationProto() { return Optional.empty(); } - - //Override default behavior to give ContainerId to AM - default Optional allocateContainerId(Configuration conf) { return Optional.empty(); } - - //Whether this framework requires addition of the default Yarn ServicePlugins - default boolean isUsingYarnServicePlugin() { - return true; - } - - //Whether to check task resources against ClusterInfo - default boolean checkTaskResources(Map vertices, AppContext appContext) { return true; } - - default Optional> getSessionToken( - ApplicationAttemptId appAttemptID, - JobTokenSecretManager jobTokenSecretManager, - Credentials amCredentials - ) { return Optional.empty(); } - - default Optional getAdditionalSessionResources(String dir) { - return Optional.empty(); - } -} diff --git a/tez-dag/src/main/java/org/apache/tez/frameworkplugins/ServerFrameworkService.java b/tez-dag/src/main/java/org/apache/tez/frameworkplugins/ServerFrameworkService.java index f776d4eb9d..5b19130a53 100644 --- a/tez-dag/src/main/java/org/apache/tez/frameworkplugins/ServerFrameworkService.java +++ b/tez-dag/src/main/java/org/apache/tez/frameworkplugins/ServerFrameworkService.java @@ -6,9 +6,9 @@ * 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 - * + *

    + * 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. @@ -17,16 +17,17 @@ */ package org.apache.tez.frameworkplugins; -import java.util.Optional; - import org.apache.hadoop.conf.Configuration; import org.apache.tez.client.registry.AMRegistry; -/* - FrameworkService that runs code within the AM process launched from DAGAppMaster.main(..) - Bundles together an AMRegistry and AmExtensions impl. that are compatible +/** + * A {@code FrameworkService} that runs inside the Application Master (AM) process. + * + *

    This service bundles together an {@code AMRegistry} and an + * {@code AMExtensions} implementation that are designed to be compatible and + * work together within the AM lifecycle.

    */ public interface ServerFrameworkService extends FrameworkService { - Optional createOrGetAMRegistry(Configuration conf); - Optional createOrGetDAGAppMasterExtensions(); + AMRegistry getAMRegistry(Configuration conf); + AMExtensions getAMExtensions(); } diff --git a/tez-dag/src/main/java/org/apache/tez/frameworkplugins/TaskResourceException.java b/tez-dag/src/main/java/org/apache/tez/frameworkplugins/TaskResourceException.java new file mode 100644 index 0000000000..697c2a7697 --- /dev/null +++ b/tez-dag/src/main/java/org/apache/tez/frameworkplugins/TaskResourceException.java @@ -0,0 +1,24 @@ +/** + * 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.tez.frameworkplugins; + +public class TaskResourceException extends Exception { + public TaskResourceException(String msg) { + super(msg); + } +} diff --git a/tez-dag/src/main/java/org/apache/tez/frameworkplugins/package-info.java b/tez-dag/src/main/java/org/apache/tez/frameworkplugins/package-info.java new file mode 100644 index 0000000000..8d75aa3614 --- /dev/null +++ b/tez-dag/src/main/java/org/apache/tez/frameworkplugins/package-info.java @@ -0,0 +1,21 @@ +/** + * 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. + */ +/** + * Provides basic classes of framework plugins for Apache Tez. + */ +package org.apache.tez.frameworkplugins; diff --git a/tez-dag/src/main/java/org/apache/tez/frameworkplugins/yarn/YarnServerFrameworkService.java b/tez-dag/src/main/java/org/apache/tez/frameworkplugins/yarn/YarnServerFrameworkService.java new file mode 100644 index 0000000000..2ca4110932 --- /dev/null +++ b/tez-dag/src/main/java/org/apache/tez/frameworkplugins/yarn/YarnServerFrameworkService.java @@ -0,0 +1,120 @@ +/** + * 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.tez.frameworkplugins.yarn; + +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.util.Map; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.Credentials; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.yarn.api.ApplicationConstants; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.util.ConverterUtils; +import org.apache.tez.client.registry.AMRegistry; +import org.apache.tez.common.TezUtilsInternal; +import org.apache.tez.common.security.JobTokenIdentifier; +import org.apache.tez.common.security.JobTokenSecretManager; +import org.apache.tez.common.security.TokenCache; +import org.apache.tez.dag.api.TezConstants; +import org.apache.tez.dag.api.records.DAGProtos; +import org.apache.tez.dag.app.ClusterInfo; +import org.apache.tez.dag.app.dag.Vertex; +import org.apache.tez.frameworkplugins.AMExtensions; +import org.apache.tez.frameworkplugins.ServerFrameworkService; +import org.apache.tez.frameworkplugins.TaskResourceException; + +/** + * YARN-based server framework service implementation. + * Provides default YARN framework server functionality with default implementations + * for all AmExtensions methods. + */ +public class YarnServerFrameworkService implements ServerFrameworkService { + + private final YarnAMExtensions amExtensions = new YarnAMExtensions(); + + @Override + public AMRegistry getAMRegistry(Configuration conf) { + // YARN mode doesn't require a custom AM registry + return null; + } + + @Override + public AMExtensions getAMExtensions() { + return amExtensions; + } + + /** + * Default YARN implementation of AmExtensions. + * Provides sensible defaults for all methods. + */ + public static class YarnAMExtensions implements AMExtensions { + + @Override + public DAGProtos.ConfigurationProto loadConfigurationProto() throws IOException { + return TezUtilsInternal + .readUserSpecifiedTezConfiguration(System.getenv(ApplicationConstants.Environment.PWD.name())); + } + + @Override + public ContainerId allocateContainerId(Configuration conf) { + String containerIdStr = System.getenv(ApplicationConstants.Environment.CONTAINER_ID.name()); + return ConverterUtils.toContainerId(containerIdStr); + } + + @Override + public void checkTaskResources(Map vertices, ClusterInfo clusterInfo) throws TaskResourceException { + Resource maxContainerCapability = clusterInfo.getMaxContainerCapability(); + for (Vertex v : vertices.values()) { + // TODO TEZ-2003 (post) TEZ-2624 Ideally, this should be per source. + if (v.getTaskResource().compareTo(maxContainerCapability) > 0) { + String msg = "Vertex's TaskResource is beyond the cluster container capability," + + "Vertex=" + v.getLogIdentifier() +", Requested TaskResource=" + v.getTaskResource() + + ", Cluster MaxContainerCapability=" + maxContainerCapability; + throw new TaskResourceException(msg); + } + } + } + + @Override + public Token getSessionToken( + ApplicationAttemptId appAttemptID, + JobTokenSecretManager jobTokenSecretManager, + Credentials amCredentials) { + return TokenCache.getSessionToken(amCredentials); + } + + @Override + public DAGProtos.PlanLocalResourcesProto getAdditionalSessionResources(String workingDirectory) throws IOException { + FileInputStream sessionResourcesStream = null; + try { + sessionResourcesStream = + new FileInputStream(new File(workingDirectory, TezConstants.TEZ_AM_LOCAL_RESOURCES_PB_FILE_NAME)); + return DAGProtos.PlanLocalResourcesProto.parseDelimitedFrom(sessionResourcesStream); + } finally { + if (sessionResourcesStream != null) { + sessionResourcesStream.close(); + } + } + } + } +} diff --git a/tez-dag/src/main/java/org/apache/tez/frameworkplugins/yarn/package-info.java b/tez-dag/src/main/java/org/apache/tez/frameworkplugins/yarn/package-info.java new file mode 100644 index 0000000000..cb50990e68 --- /dev/null +++ b/tez-dag/src/main/java/org/apache/tez/frameworkplugins/yarn/package-info.java @@ -0,0 +1,21 @@ +/** + * 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. + */ +/** + * Provides YARN-specific framework plugins for Apache Tez. + */ +package org.apache.tez.frameworkplugins.yarn; diff --git a/tez-dag/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZkStandaloneAmExtensions.java b/tez-dag/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZkStandaloneAMExtensions.java similarity index 53% rename from tez-dag/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZkStandaloneAmExtensions.java rename to tez-dag/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZkStandaloneAMExtensions.java index 6a86914bb2..ff3af7b495 100644 --- a/tez-dag/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZkStandaloneAmExtensions.java +++ b/tez-dag/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZkStandaloneAMExtensions.java @@ -6,9 +6,9 @@ * 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 - * + *

    + * 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. @@ -19,7 +19,6 @@ import java.io.IOException; import java.util.Map; -import java.util.Optional; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.Text; @@ -34,71 +33,58 @@ import org.apache.tez.common.security.JobTokenSecretManager; import org.apache.tez.common.security.TokenCache; import org.apache.tez.dag.api.records.DAGProtos; -import org.apache.tez.dag.app.AppContext; +import org.apache.tez.dag.app.ClusterInfo; import org.apache.tez.dag.app.dag.Vertex; -import org.apache.tez.frameworkplugins.AmExtensions; +import org.apache.tez.frameworkplugins.AMExtensions; import org.apache.tez.frameworkplugins.ServerFrameworkService; -public class ZkStandaloneAmExtensions implements AmExtensions { +public class ZkStandaloneAMExtensions implements AMExtensions { - ServerFrameworkService myFrameworkService; + private final ServerFrameworkService frameworkService; - public ZkStandaloneAmExtensions(ServerFrameworkService myFrameworkService) { - this.myFrameworkService = myFrameworkService; + public ZkStandaloneAMExtensions(ServerFrameworkService frameworkService) { + this.frameworkService = frameworkService; } - @Override public Optional allocateContainerId(Configuration conf) { + @Override + public ContainerId allocateContainerId(Configuration conf) { try { - Optional amRegistry = myFrameworkService.createOrGetAMRegistry(conf); - if(amRegistry.isPresent()) { - ApplicationId appId = amRegistry.get().generateNewId().get(); - // attempId is set to 1 only then APP_LAUNCHED event gets triggered + AMRegistry amRegistry = frameworkService.getAMRegistry(conf); + if (amRegistry != null) { + ApplicationId appId = amRegistry.generateNewId(); + // attemptId is set to 1 only then APP_LAUNCHED event gets triggered ApplicationAttemptId applicationAttemptId = ApplicationAttemptId.newInstance(appId, 1); - return Optional.of(ContainerId.newContainerId(applicationAttemptId, 0)); + return ContainerId.newContainerId(applicationAttemptId, 0); } else { throw new RuntimeException("AMRegistry is required for ZkStandaloneAmExtensions"); } - - } catch(Exception e) { + } catch (Exception e) { throw new RuntimeException(e); } } - @Override public boolean checkTaskResources(Map vertices, AppContext appContext) { - return false; - } - @Override - public boolean isUsingYarnServicePlugin() { - return false; + public void checkTaskResources(Map vertices, ClusterInfo clusterInfo) { + // no-op: Zookeeper-based framework current enforce task resources at the moment } - @Override public Optional loadConfigurationProto() { - try { - return Optional.of(TezUtilsInternal.loadConfProtoFromText()); - } catch (IOException e) { - throw new RuntimeException(e); - } + @Override + public DAGProtos.ConfigurationProto loadConfigurationProto() throws IOException { + return TezUtilsInternal.loadConfProtoFromText(); } @Override - public Optional> getSessionToken( - ApplicationAttemptId appAttemptID, - JobTokenSecretManager jobTokenSecretManager, - Credentials amCredentials - ) { - JobTokenIdentifier identifier = new JobTokenIdentifier(new Text( - appAttemptID.getApplicationId().toString())); - Token newSessionToken = new Token(identifier, - jobTokenSecretManager); + public Token getSessionToken(ApplicationAttemptId appAttemptID, + JobTokenSecretManager jobTokenSecretManager, Credentials amCredentials) { + JobTokenIdentifier identifier = new JobTokenIdentifier(new Text(appAttemptID.getApplicationId().toString())); + Token newSessionToken = new Token<>(identifier, jobTokenSecretManager); newSessionToken.setService(identifier.getJobId()); TokenCache.setSessionToken(newSessionToken, amCredentials); - return Optional.of(newSessionToken); + return newSessionToken; } @Override - public Optional getAdditionalSessionResources(String dir) { - return Optional.of(DAGProtos.PlanLocalResourcesProto.getDefaultInstance()); + public DAGProtos.PlanLocalResourcesProto getAdditionalSessionResources(String workingDirectory) { + return DAGProtos.PlanLocalResourcesProto.getDefaultInstance(); } - } diff --git a/tez-dag/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZkStandaloneServerFrameworkService.java b/tez-dag/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZkStandaloneServerFrameworkService.java new file mode 100644 index 0000000000..dc8b7b91df --- /dev/null +++ b/tez-dag/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZkStandaloneServerFrameworkService.java @@ -0,0 +1,55 @@ +/** + * 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.tez.frameworkplugins.zookeeper; + + +import org.apache.hadoop.conf.Configuration; +import org.apache.tez.client.registry.AMRegistry; +import org.apache.tez.dag.api.TezConstants; +import org.apache.tez.dag.api.client.registry.zookeeper.ZkAMRegistry; +import org.apache.tez.frameworkplugins.AMExtensions; +import org.apache.tez.frameworkplugins.ServerFrameworkService; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ZkStandaloneServerFrameworkService implements ServerFrameworkService { + private static final Logger LOG = LoggerFactory.getLogger(ZkStandaloneServerFrameworkService.class); + private ZkAMRegistry amRegistry; + + @Override + public synchronized AMRegistry getAMRegistry(Configuration conf) { + if (amRegistry == null) { + try { + final String externalID = System.getenv(TezConstants.TEZ_AM_EXTERNAL_ID); + amRegistry = new ZkAMRegistry(externalID); + amRegistry.init(conf); + amRegistry.start(); + LOG.info("Created Zookeeper based AM Registry with externalID: {}", externalID); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + return amRegistry; + } + + @Override + public AMExtensions getAMExtensions() { + return new ZkStandaloneAMExtensions(this); + } +} diff --git a/tez-dag/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZookeeperStandaloneServerFrameworkService.java b/tez-dag/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZookeeperStandaloneServerFrameworkService.java deleted file mode 100644 index 877ed64cf7..0000000000 --- a/tez-dag/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZookeeperStandaloneServerFrameworkService.java +++ /dev/null @@ -1,39 +0,0 @@ -package org.apache.tez.frameworkplugins.zookeeper; - -import java.util.Optional; - -import org.apache.hadoop.conf.Configuration; -import org.apache.tez.client.registry.AMRegistry; -import org.apache.tez.dag.api.TezConstants; -import org.apache.tez.dag.api.client.registry.zookeeper.ZkAMRegistry; -import org.apache.tez.frameworkplugins.AmExtensions; -import org.apache.tez.frameworkplugins.ServerFrameworkService; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class ZookeeperStandaloneServerFrameworkService implements ServerFrameworkService { - private static final Logger LOG = LoggerFactory.getLogger(ZookeeperStandaloneServerFrameworkService.class); - private ZkAMRegistry amRegistry; - - @Override - public synchronized Optional createOrGetAMRegistry(Configuration conf) { - if (amRegistry == null) { - try { - final String externalID = System.getenv(TezConstants.TEZ_AM_EXTERNAL_ID); - amRegistry = new ZkAMRegistry(externalID); - amRegistry.init(conf); - amRegistry.start(); - LOG.info("Created Zookeeper based AM Registry with externalID: {}", externalID); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - return Optional.of(amRegistry); - } - - @Override - public Optional createOrGetDAGAppMasterExtensions() { - return Optional.of(new ZkStandaloneAmExtensions(this)); - } -} diff --git a/tez-dag/src/main/java/org/apache/tez/frameworkplugins/zookeeper/package-info.java b/tez-dag/src/main/java/org/apache/tez/frameworkplugins/zookeeper/package-info.java new file mode 100644 index 0000000000..c3ddc3f7a7 --- /dev/null +++ b/tez-dag/src/main/java/org/apache/tez/frameworkplugins/zookeeper/package-info.java @@ -0,0 +1,21 @@ +/** + * 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. + */ +/** + * Provides Zookeeper-specific framework plugins for Apache Tez. + */ +package org.apache.tez.frameworkplugins.zookeeper; diff --git a/tez-dag/src/test/java/org/apache/tez/client/registry/zookeeper/TestZkAMRegistryClient.java b/tez-dag/src/test/java/org/apache/tez/client/registry/zookeeper/TestZkAMRegistryClient.java new file mode 100644 index 0000000000..69fe2341c0 --- /dev/null +++ b/tez-dag/src/test/java/org/apache/tez/client/registry/zookeeper/TestZkAMRegistryClient.java @@ -0,0 +1,197 @@ +/** + * 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.tez.client.registry.zookeeper; + +import static org.junit.Assert.*; + +import java.io.File; +import java.util.List; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.commons.io.IOUtils; +import org.apache.curator.test.TestingServer; +import org.apache.hadoop.security.Credentials; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.util.SystemClock; +import org.apache.tez.client.registry.AMRecord; +import org.apache.tez.client.registry.AMRegistryClientListener; +import org.apache.tez.dag.api.TezConfiguration; +import org.apache.tez.dag.app.DAGAppMaster; +import org.apache.tez.dag.app.MockDAGAppMaster; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Unit tests for {@link ZkAMRegistryClient}. + *

    + * This test class validates the ZooKeeper-based AM (Application Master) registry and discovery + * mechanism. It tests that when a DAGAppMaster is started with STANDALONE_ZOOKEEPER framework mode, + * it properly registers itself to ZooKeeper and can be discovered by a {@link ZkAMRegistryClient}. + *

    + *

    + * The tests use an embedded ZooKeeper {@link TestingServer} to avoid external dependencies + * and ensure test isolation. + *

    + */ +public class TestZkAMRegistryClient { + private static final Logger LOG = LoggerFactory.getLogger(TestZkAMRegistryClient.class); + private static final File TEST_DIR = new File(System.getProperty("test.build.data", "target"), + TestZkAMRegistryClient.class.getName()).getAbsoluteFile(); + + /** + * Embedded ZooKeeper server for testing. Uses Apache Curator's {@link TestingServer} + * to provide an in-memory ZooKeeper instance. + */ + private TestingServer zkServer; + + /** + * ZooKeeper-based AM registry client used to discover and retrieve AM records. + */ + private ZkAMRegistryClient registryClient; + + /** + * Mock DAGAppMaster instance that registers itself to the ZooKeeper registry. + */ + private DAGAppMaster dagAppMaster; + + @Before + public void setup() throws Exception { + zkServer = new TestingServer(); + zkServer.start(); + LOG.info("Started ZooKeeper test server on port: {}", zkServer.getPort()); + } + + @After + public void teardown() throws Exception { + if (dagAppMaster != null) { + dagAppMaster.stop(); + } + IOUtils.closeQuietly(registryClient); + IOUtils.closeQuietly(zkServer); + } + + /** + * Tests the complete ZooKeeper-based AM registry and discovery flow. + *

    + * This test validates the following workflow: + *

    + *
      + *
    1. Configure Tez with STANDALONE_ZOOKEEPER framework mode
    2. + *
    3. Create and start a {@link ZkAMRegistryClient} with an event listener
    4. + *
    5. Start a {@link MockDAGAppMaster} which registers itself to ZooKeeper
    6. + *
    7. Verify that the registry client's listener is notified of the AM registration
    8. + *
    9. Verify the AM record can be retrieved via {@link ZkAMRegistryClient#getRecord(String)}
    10. + *
    11. Verify the AM appears in the list from {@link ZkAMRegistryClient#getAllRecords()}
    12. + *
    13. Validate all expected fields (host, port, applicationId) are correctly set
    14. + *
    + *

    + * The test uses a {@link CountDownLatch} to synchronize between the AM registration + * event and the test assertions, ensuring the AM has fully registered before validation. + *

    + * + * @throws Exception if any part of the test fails + */ + @Test(timeout = 10000) + public void testZkAmRegistryDiscovery() throws Exception { + TezConfiguration tezConf = getTezConfForZkDiscovery(); + + ApplicationId appId = ApplicationId.newInstance(System.currentTimeMillis(), 1); + ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(appId, 1); + ContainerId containerId = ContainerId.newContainerId(attemptId, 1); + + CountDownLatch amRegisteredLatch = new CountDownLatch(1); + AtomicBoolean amDiscovered = new AtomicBoolean(false); + + // Create and start the ZkAMRegistryClient + registryClient = ZkAMRegistryClient.getClient(tezConf); + registryClient.addListener(new AMRegistryClientListener() { + @Override + public void onAdd(AMRecord amRecord) { + LOG.info("AM added to registry: {}", amRecord); + if (amRecord.getApplicationId().equals(appId)) { + amDiscovered.set(true); + amRegisteredLatch.countDown(); + } + } + + @Override + public void onRemove(AMRecord amRecord) { + LOG.info("AM removed from registry: {}", amRecord); + } + }); + registryClient.start(); + + String workingDir = TEST_DIR.toString(); + String[] localDirs = new String[]{TEST_DIR.toString()}; + String[] logDirs = new String[]{TEST_DIR + "/logs"}; + String jobUserName = UserGroupInformation.getCurrentUser().getShortUserName(); + + dagAppMaster = new MockDAGAppMaster(attemptId, containerId, "localhost", 0, 0, SystemClock.getInstance(), + System.currentTimeMillis(), true, workingDir, localDirs, logDirs, new AtomicBoolean(true), false, false, + new Credentials(), jobUserName, 1, 1); + + dagAppMaster.init(tezConf); + dagAppMaster.start(); + + // Wait for AM to be registered in ZooKeeper + boolean registered = amRegisteredLatch.await(30, TimeUnit.SECONDS); + assertTrue("AM was not registered in ZooKeeper within timeout", registered); + assertTrue("AM was not discovered by registry client", amDiscovered.get()); + + // Verify the AM record is available through the registry client + AMRecord amRecord = registryClient.getRecord(appId.toString()); + assertNotNull("AM record should be retrievable from registry", amRecord); + assertEquals("Application ID should match", appId, amRecord.getApplicationId()); + assertNotNull("Host should be set", amRecord.getHost()); + assertTrue("Port should be positive", amRecord.getPort() > 0); + + // Verify getAllRecords also returns the AM + List allRecords = registryClient.getAllRecords(); + assertNotNull("getAllRecords should not return null", allRecords); + assertFalse("getAllRecords should contain at least one record", allRecords.isEmpty()); + + boolean found = false; + for (AMRecord record : allRecords) { + if (record.getApplicationId().equals(appId)) { + found = true; + break; + } + } + assertTrue("AM record should be in getAllRecords", found); + LOG.info("Test completed successfully. AM was discovered: {}", amRecord); + } + + private TezConfiguration getTezConfForZkDiscovery() { + TezConfiguration tezConf = new TezConfiguration(); + tezConf.set(TezConfiguration.TEZ_FRAMEWORK_MODE, "STANDALONE_ZOOKEEPER"); + tezConf.set(TezConfiguration.TEZ_AM_ZOOKEEPER_QUORUM, "localhost:" + zkServer.getPort()); + tezConf.setBoolean(TezConfiguration.TEZ_LOCAL_MODE, true); + tezConf.set(TezConfiguration.TEZ_AM_STAGING_DIR, TEST_DIR.toString()); + return tezConf; + } +} diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestCommit.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestCommit.java index 953b473ab9..863abffc26 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestCommit.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestCommit.java @@ -105,6 +105,8 @@ import org.apache.tez.dag.records.TezTaskAttemptID; import org.apache.tez.dag.records.TezTaskID; import org.apache.tez.dag.records.TezVertexID; +import org.apache.tez.frameworkplugins.AMExtensions; +import org.apache.tez.frameworkplugins.yarn.YarnServerFrameworkService; import org.apache.tez.hadoop.shim.DefaultHadoopShim; import org.apache.tez.runtime.api.OutputCommitter; import org.apache.tez.runtime.api.OutputCommitterContext; @@ -155,6 +157,7 @@ public class TestCommit { private ExecutorService rawExecutor; private ListeningExecutorService execService; + private AMExtensions amExtensions = new YarnServerFrameworkService.YarnAMExtensions(); private class DagEventDispatcher implements EventHandler { @Override @@ -306,6 +309,7 @@ public void setupDAG(DAGPlan dagPlan) { fsTokens = new Credentials(); appContext = mock(AppContext.class); when(appContext.getHadoopShim()).thenReturn(new DefaultHadoopShim()); + when(appContext.getAmExtensions()).thenReturn(amExtensions); rawExecutor = Executors.newCachedThreadPool(new ThreadFactoryBuilder() .setDaemon(true).setNameFormat("App Shared Pool - " + "#%d").build()); execService = MoreExecutors.listeningDecorator(rawExecutor); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java index c0bac853c1..92d8a15d9e 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java @@ -140,6 +140,8 @@ import org.apache.tez.dag.records.TezTaskAttemptID; import org.apache.tez.dag.records.TezTaskID; import org.apache.tez.dag.records.TezVertexID; +import org.apache.tez.frameworkplugins.AMExtensions; +import org.apache.tez.frameworkplugins.yarn.YarnServerFrameworkService; import org.apache.tez.hadoop.shim.DefaultHadoopShim; import org.apache.tez.hadoop.shim.HadoopShim; import org.apache.tez.runtime.api.OutputCommitter; @@ -210,6 +212,7 @@ public class TestDAGImpl { private TaskAttemptEventDispatcher taskAttemptEventDispatcher; private ClusterInfo clusterInfo = new ClusterInfo(Resource.newInstance(8192,10)); private HadoopShim defaultShim = new DefaultHadoopShim(); + private AMExtensions amExtensions = new YarnServerFrameworkService.YarnAMExtensions(); static { Limits.reset(); @@ -872,6 +875,7 @@ public void setup() { final ListenableFuture mockFuture = mock(ListenableFuture.class); when(appContext.getHadoopShim()).thenReturn(defaultShim); when(appContext.getApplicationID()).thenReturn(appAttemptId.getApplicationId()); + doReturn(amExtensions).when(appContext).getAmExtensions(); doAnswer(new Answer() { public ListenableFuture answer(InvocationOnMock invocation) { @@ -902,6 +906,7 @@ public ListenableFuture answer(InvocationOnMock invocation) { doReturn(aclManager).when(mrrAppContext).getAMACLManager(); doReturn(execService).when(mrrAppContext).getExecService(); doReturn(defaultShim).when(mrrAppContext).getHadoopShim(); + doReturn(amExtensions).when(mrrAppContext).getAmExtensions(); mrrDagId = TezDAGID.getInstance(appAttemptId.getApplicationId(), 2); mrrDagPlan = createTestMRRDAGPlan(); @@ -935,6 +940,7 @@ public ListenableFuture answer(InvocationOnMock invocation) { .when(groupAppContext).getApplicationID(); doReturn(historyEventHandler).when(groupAppContext).getHistoryHandler(); doReturn(clusterInfo).when(groupAppContext).getClusterInfo(); + doReturn(amExtensions).when(groupAppContext).getAmExtensions(); // reset totalCommitCounter to 0 TotalCountingOutputCommitter.totalCommitCounter = 0; @@ -1005,6 +1011,7 @@ private void setupDAGWithCustomEdge(ExceptionLocation exLocation, boolean useLeg doReturn(appAttemptId.getApplicationId()).when(dagWithCustomEdgeAppContext).getApplicationID(); doReturn(historyEventHandler).when(dagWithCustomEdgeAppContext).getHistoryHandler(); doReturn(clusterInfo).when(dagWithCustomEdgeAppContext).getClusterInfo(); + doReturn(amExtensions).when(dagWithCustomEdgeAppContext).getAmExtensions(); dispatcher.register(TaskAttemptEventType.class, new TaskAttemptEventDisptacher2()); dispatcher.register(AMSchedulerEventType.class, new AMSchedulerEventHandler()); when(dagWithCustomEdgeAppContext.getContainerLauncherName(anyInt())).thenReturn( diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java index ee3684ae03..8b82a6aa3e 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java @@ -116,6 +116,7 @@ import org.apache.tez.dag.records.TezTaskAttemptID; import org.apache.tez.dag.records.TezTaskID; import org.apache.tez.dag.records.TezVertexID; +import org.apache.tez.frameworkplugins.yarn.YarnServerFrameworkService; import org.apache.tez.hadoop.shim.DefaultHadoopShim; import org.apache.tez.runtime.api.Event; import org.apache.tez.runtime.api.InputInitializer; @@ -325,6 +326,7 @@ public void setup() { when(appContext.getHadoopShim()).thenReturn(new DefaultHadoopShim()); when(appContext.getApplicationID()).thenReturn(appAttemptId.getApplicationId()); when(appContext.getClock()).thenReturn(new SystemClock()); + when(appContext.getAmExtensions()).thenReturn(new YarnServerFrameworkService.YarnAMExtensions()); doAnswer(new Answer() { public ListenableFuture answer(InvocationOnMock invocation) { diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl2.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl2.java index c29a471604..a3b870ff3a 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl2.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl2.java @@ -373,13 +373,15 @@ public ExecutionContextTestInfoHolder(VertexExecutionContext vertexExecutionCont this.vertexName = "testvertex"; this.vertexExecutionContext = vertexExecutionContext; this.defaultExecutionContext = defaultDagExecitionContext; + + UserPayload defaultPayload; + try { + defaultPayload = TezUtils.createUserPayloadFromConf(new Configuration(false)); + } catch (IOException e) { + throw new TezUncheckedException(e); + } + if (numPlugins == 0) { // Add default container plugins only - UserPayload defaultPayload; - try { - defaultPayload = TezUtils.createUserPayloadFromConf(new Configuration(false)); - } catch (IOException e) { - throw new TezUncheckedException(e); - } PluginManager.parsePlugin(Lists.newLinkedList(), taskSchedulers, null, true, false, defaultPayload); PluginManager.parsePlugin(Lists.newLinkedList(), containerLaunchers, null, true, false, defaultPayload); PluginManager.parsePlugin(Lists.newLinkedList(), taskComms, null, true, false, defaultPayload); @@ -402,9 +404,10 @@ public ExecutionContextTestInfoHolder(VertexExecutionContext vertexExecutionCont DAGProtos.TezEntityDescriptorProto.newBuilder() .setClassName(append(TASK_COMM_NAME_BASE, i))).build()); } - PluginManager.parsePlugin(Lists.newLinkedList(), taskSchedulers, schedulerList, false, false, null); - PluginManager.parsePlugin(Lists.newLinkedList(), containerLaunchers, launcherList, false, false, null); - PluginManager.parsePlugin(Lists.newLinkedList(), taskComms, taskCommList, false, false, null); + PluginManager.parsePlugin(Lists.newLinkedList(), taskSchedulers, schedulerList, false, false, defaultPayload); + PluginManager.parsePlugin(Lists.newLinkedList(), containerLaunchers, launcherList, false, false, + defaultPayload); + PluginManager.parsePlugin(Lists.newLinkedList(), taskComms, taskCommList, false, false, defaultPayload); } this.appContext = createDefaultMockAppContext(); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexManager.java index 8778f2b4e9..97ba83652d 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexManager.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexManager.java @@ -277,6 +277,7 @@ public void testSendCustomProcessorEvent() throws Exception { verify(mockHandler, times(1)).handle(requestCaptor.capture()); CustomProcessorEvent cpe = (CustomProcessorEvent)(requestCaptor.getValue().getEvents().get(0).getEvent()); + // should be able to get payload any times for (int i = 0; i < 2; i++) { ByteBuffer payloadBuffer = cpe.getPayload(); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java index 0c808af0b6..277291110c 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerHelpers.java @@ -597,6 +597,7 @@ static TaskSchedulerContext setupMockTaskSchedulerContext(String appHost, int ap throw new TezUncheckedException(e); } when(mockContext.getInitialUserPayload()).thenReturn(userPayload); + when(mockContext.isSession()).thenReturn(isSession); if (containerSignatureMatcher != null) { when(mockContext.getContainerSignatureMatcher()) .thenReturn(containerSignatureMatcher); diff --git a/tez-examples/pom.xml b/tez-examples/pom.xml index 18429a459a..7d303d762c 100644 --- a/tez-examples/pom.xml +++ b/tez-examples/pom.xml @@ -71,6 +71,10 @@ commons-cli commons-cli
    + + org.apache.curator + curator-test + junit junit diff --git a/tez-examples/src/main/java/org/apache/tez/examples/ExampleBase.java b/tez-examples/src/main/java/org/apache/tez/examples/ExampleBase.java deleted file mode 100644 index e9c6cb69e3..0000000000 --- a/tez-examples/src/main/java/org/apache/tez/examples/ExampleBase.java +++ /dev/null @@ -1,40 +0,0 @@ -package org.apache.tez.examples; - -import java.io.IOException; - -import org.apache.tez.client.TezClient; -import org.apache.tez.dag.api.DAG; -import org.apache.tez.dag.api.TezConfiguration; -import org.apache.tez.dag.api.TezException; - -import org.slf4j.Logger; - -public abstract class ExampleBase { - /** - * Validate the arguments - * - * @param otherArgs arguments, if any - * @return Zero indicates success, non-zero indicates failure - */ - protected abstract int validateArgs(String[] otherArgs); - - /** - * Print usage instructions for this example - */ - protected abstract void printUsage(); - - /** - * Create and execute the actual DAG for the example - * - * @param args arguments for execution - * @param tezConf the tez configuration instance to be used while processing the DAG - * @param tezClient the tez client instance to use to run the DAG if any custom monitoring is - * required. Otherwise the utility method {@link #runDag(DAG, - * boolean, Logger)} should be used - * @return Zero indicates success, non-zero indicates failure - * @throws IOException - * @throws TezException - */ - protected abstract int runJob(String[] args, TezConfiguration tezConf, - TezClient tezClient) throws Exception; -} diff --git a/tez-examples/src/main/java/org/apache/tez/examples/StandaloneWordCount.java b/tez-examples/src/main/java/org/apache/tez/examples/StandaloneWordCount.java deleted file mode 100644 index 47a8c75736..0000000000 --- a/tez-examples/src/main/java/org/apache/tez/examples/StandaloneWordCount.java +++ /dev/null @@ -1,128 +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.tez.examples; - -import java.io.IOException; - -import org.apache.hadoop.io.IntWritable; -import org.apache.hadoop.io.Text; -import org.apache.tez.client.TezClient; -import org.apache.tez.dag.api.*; -import org.apache.tez.mapreduce.SumProcessor; -import org.apache.tez.mapreduce.TokenProcessor; -import org.apache.tez.runtime.api.*; -import org.apache.tez.runtime.library.FakeInput; -import org.apache.tez.runtime.library.FakeInputInitializer; -import org.apache.tez.runtime.library.FakeOutput; -import org.apache.tez.runtime.library.FakeOutputCommitter; -import org.apache.tez.runtime.library.conf.OrderedPartitionedKVEdgeConfig; -import org.apache.tez.runtime.library.partitioner.HashPartitioner; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Simple example to perform WordCount using Tez API's. WordCount is the - * HelloWorld program of distributed data processing and counts the number - * of occurrences of a word in a distributed text data set. - */ -public class StandaloneWordCount extends ExampleBase { - - private static final Logger LOG = LoggerFactory.getLogger(StandaloneWordCount.class); - static String INPUT = "Input"; - static String SUMMATION = "Summation"; - static String OUTPUT = "Output"; - static String TOKENIZER = "Tokenizer"; - - /* - * Example code to write a processor in Tez. - * Processors typically apply the main application logic to the data. - * TokenProcessor tokenizes the input data. - * It uses an input that provide a Key-Value reader and writes - * output to a Key-Value writer. The processor inherits from SimpleProcessor - * since it does not need to handle any advanced constructs for Processors. - */ - - /* - * Example code to write a processor that commits final output to a data sink - * The SumProcessor aggregates the sum of individual word counts generated by - * the TokenProcessor. - * The SumProcessor is connected to a DataSink. In this case, its an Output that - * writes the data via an OutputFormat to a data sink (typically HDFS). Thats why - * it derives from SimpleMRProcessor that takes care of handling the necessary - * output commit operations that makes the final output available for consumers. - */ - - private DAG createDAG(TezConfiguration tezConf, String inputPath, String outputPath, - int numPartitions) throws IOException { - - InputDescriptor inputDescriptor = InputDescriptor.create(FakeInput.class.getName()); - InputInitializerDescriptor inputInitializerDescriptor = - InputInitializerDescriptor.create(FakeInputInitializer.class.getName()); - DataSourceDescriptor dataSource = - DataSourceDescriptor.create(inputDescriptor, inputInitializerDescriptor, null); - - OutputDescriptor outputDescriptor = OutputDescriptor.create(FakeOutput.class.getName()); - OutputCommitterDescriptor outputCommitterDescriptor = - OutputCommitterDescriptor.create(FakeOutputCommitter.class.getName()); - DataSinkDescriptor dataSink = - DataSinkDescriptor.create(outputDescriptor, outputCommitterDescriptor, null); - - Vertex tokenizerVertex = Vertex.create(TOKENIZER, ProcessorDescriptor.create( - TokenProcessor.class.getName())).addDataSource(INPUT, dataSource); - - OrderedPartitionedKVEdgeConfig edgeConf = OrderedPartitionedKVEdgeConfig - .newBuilder(Text.class.getName(), IntWritable.class.getName(), - HashPartitioner.class.getName()) - .setFromConfiguration(tezConf) - .build(); - - Vertex summationVertex = Vertex.create(SUMMATION, - ProcessorDescriptor.create(SumProcessor.class.getName()), numPartitions) - .addDataSink(OUTPUT, dataSink); - - DAG dag = DAG.create("WordCount"); - dag.addVertex(tokenizerVertex) - .addVertex(summationVertex) - .addEdge( - Edge.create(tokenizerVertex, summationVertex, edgeConf.createDefaultEdgeProperty())); - return dag; - } - - @Override - protected void printUsage() { - System.err.println("Usage: " + " wordcount in out"); - } - - @Override - protected int validateArgs(String[] otherArgs) { - if (otherArgs.length != 2 ) { - return 2; - } - return 0; - } - - @Override - protected int runJob(String[] args, TezConfiguration tezConf, - TezClient tezClient) throws Exception { - DAG dag = createDAG(tezConf, null, null,1); - LOG.info("Running WordCount"); - return StandaloneWordCountDriver.runDag(dag, tezClient.getAppMasterApplicationId() , LOG); - } - -} diff --git a/tez-examples/src/main/java/org/apache/tez/examples/StandaloneWordCountDriver.java b/tez-examples/src/main/java/org/apache/tez/examples/StandaloneWordCountDriver.java deleted file mode 100644 index e58760a7d2..0000000000 --- a/tez-examples/src/main/java/org/apache/tez/examples/StandaloneWordCountDriver.java +++ /dev/null @@ -1,109 +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.tez.examples; - -import java.io.IOException; -import java.util.Collections; -import java.util.List; -import java.util.Set; - -import org.apache.hadoop.classification.InterfaceAudience.Private; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.security.Credentials; -import org.apache.hadoop.security.token.Token; -import org.apache.hadoop.security.token.TokenIdentifier; -import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.tez.client.CallerContext; -import org.apache.tez.client.TezClient; -import org.apache.tez.client.registry.AMRecord; -import org.apache.tez.client.registry.zookeeper.ZkAMRegistryClient; -import org.apache.tez.dag.api.DAG; -import org.apache.tez.dag.api.TezConfiguration; -import org.apache.tez.dag.api.TezException; -import org.apache.tez.dag.api.client.DAGClient; -import org.apache.tez.dag.api.client.DAGStatus; -import org.apache.tez.dag.api.client.StatusGetOpts; - -import com.google.common.collect.Sets; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -@Private -public abstract class StandaloneWordCountDriver { - - private static final Logger LOG = LoggerFactory.getLogger(StandaloneWordCountDriver.class); - - private static TezClient tezClientInternal; - - public static void main(String[] args) throws Exception { - ExampleBase clazz = new StandaloneWordCount(); - _execute(clazz, args, null, null); - } - - private static int _execute(ExampleBase clazz, String[] otherArgs, TezConfiguration tezConf, TezClient tezClient) throws - Exception { - tezConf = new TezConfiguration(); - tezConf.set(TezConfiguration.TEZ_AM_ZOOKEEPER_QUORUM, "localhost:2181"); - ZkAMRegistryClient registryClientZk = ZkAMRegistryClient.getClient(tezConf); - registryClientZk.start(); - List sessions = registryClientZk.getAllRecords(); - Collections.shuffle(sessions); - AMRecord am = sessions.get(0); - tezConf.set(TezConfiguration.TEZ_FRAMEWORK_MODE, "STANDALONE_ZOOKEEPER"); - tezConf.setBoolean(TezConfiguration.TEZ_IGNORE_LIB_URIS, true); - tezClientInternal = createTezClient(am.getApplicationId().toString(), tezConf); - StandaloneWordCount standaloneWordCount = new StandaloneWordCount(); - return standaloneWordCount.runJob(otherArgs, tezConf, tezClientInternal); - } - - public static int runDag(DAG dag, ApplicationId appId, Logger logger) throws Exception { - //tezClientInternal.waitTillReady(); - - CallerContext callerContext = CallerContext.create("TezExamples", - "Tez Example DAG: " + dag.getName()); - - if (appId != null) { - callerContext.setCallerIdAndType(appId.toString(), "TezExampleApplication"); - } - dag.setCallerContext(callerContext); - - DAGClient dagClient = tezClientInternal.submitDAG(dag); - Set getOpts = Sets.newHashSet(); - getOpts.add(StatusGetOpts.GET_COUNTERS); - - DAGStatus dagStatus; - dagStatus = dagClient.waitForCompletionWithStatusUpdates(getOpts); - - if (dagStatus.getState() != DAGStatus.State.SUCCEEDED) { - logger.info("DAG diagnostics: " + dagStatus.getDiagnostics()); - return -1; - } - return 0; - } - - private static TezClient createTezClient(String appId, TezConfiguration tezConf) throws IOException, TezException { - Credentials credentials = new Credentials(); - Token token = new Token(); - credentials.addToken(new Text("root"), token); - TezClient tezClient = TezClient.create("TezExampleApplication", tezConf, true, null, credentials); - return tezClient.getClient(appId); - } - -} diff --git a/tez-examples/src/main/java/org/apache/tez/examples/TezExampleBase.java b/tez-examples/src/main/java/org/apache/tez/examples/TezExampleBase.java index 507dc01e11..5716a0a0f6 100644 --- a/tez-examples/src/main/java/org/apache/tez/examples/TezExampleBase.java +++ b/tez-examples/src/main/java/org/apache/tez/examples/TezExampleBase.java @@ -123,7 +123,7 @@ public final int run(String[] args) throws Exception { } hadoopShim = new HadoopShimsLoader(conf).getHadoopShim(); - return _execute(otherArgs, null, null); + return execute(otherArgs, null, null); } /** @@ -160,7 +160,7 @@ public int run(TezConfiguration conf, String[] args, @Nullable TezClient tezClie generateSplitInClient = true; } String[] otherArgs = optionParser.getRemainingArgs(); - return _execute(otherArgs, conf, tezClient); + return execute(otherArgs, conf, tezClient); } /** @@ -215,8 +215,7 @@ private int _validateArgs(String[] args) { return 0; } - private int _execute(String[] otherArgs, TezConfiguration tezConf, TezClient tezClient) throws - Exception { + protected int execute(String[] otherArgs, TezConfiguration tezConf, TezClient tezClient) throws Exception { int result = _validateArgs(otherArgs); if (result != 0) { @@ -250,7 +249,7 @@ private int _execute(String[] otherArgs, TezConfiguration tezConf, TezClient tez } } - private TezClient createTezClient(TezConfiguration tezConf) throws IOException, TezException { + protected TezClient createTezClient(TezConfiguration tezConf) throws IOException, TezException { TezClient tezClient = TezClient.create("TezExampleApplication", tezConf); if(reconnectAppId != null) { ApplicationId appId = TezClient.appIdfromString(reconnectAppId); diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/SumProcessor.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/SumProcessor.java deleted file mode 100644 index 2054f19233..0000000000 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/SumProcessor.java +++ /dev/null @@ -1,40 +0,0 @@ -package org.apache.tez.mapreduce; - -import org.apache.hadoop.io.IntWritable; -import org.apache.hadoop.io.Text; -import org.apache.tez.mapreduce.processor.SimpleMRProcessor; -import org.apache.tez.runtime.api.ProcessorContext; -import org.apache.tez.runtime.library.api.KeyValueWriter; -import org.apache.tez.runtime.library.api.KeyValuesReader; - -import com.google.common.base.Preconditions; - -public class SumProcessor extends SimpleMRProcessor { - public SumProcessor(ProcessorContext context) { - super(context); - } - - static String OUTPUT = "Output"; - static String TOKENIZER = "Tokenizer"; - - @Override - public void run() throws Exception { - Preconditions.checkArgument(getInputs().size() == 1); - Preconditions.checkArgument(getOutputs().size() == 1); - KeyValueWriter kvWriter = (KeyValueWriter) getOutputs().get(OUTPUT).getWriter(); - // The KeyValues reader provides all values for a given key. The aggregation of values per key - // is done by the LogicalInput. Since the key is the word and the values are its counts in - // the different TokenProcessors, summing all values per key provides the sum for that word. - KeyValuesReader kvReader = (KeyValuesReader) getInputs().get(TOKENIZER).getReader(); - while (kvReader.next()) { - Text word = (Text) kvReader.getCurrentKey(); - int sum = 0; - for (Object value : kvReader.getCurrentValues()) { - sum += ((IntWritable) value).get(); - } - kvWriter.write(word, new IntWritable(sum)); - } - // deriving from SimpleMRProcessor takes care of committing the output - // It automatically invokes the commit logic for the OutputFormat if necessary. - } -} diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/TokenProcessor.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/TokenProcessor.java deleted file mode 100644 index f35f24545e..0000000000 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/TokenProcessor.java +++ /dev/null @@ -1,45 +0,0 @@ -package org.apache.tez.mapreduce; - -import java.util.StringTokenizer; - -import org.apache.hadoop.io.IntWritable; -import org.apache.hadoop.io.Text; -import org.apache.tez.runtime.api.ProcessorContext; -import org.apache.tez.runtime.library.api.KeyValueReader; -import org.apache.tez.runtime.library.api.KeyValueWriter; -import org.apache.tez.runtime.library.processor.SimpleProcessor; - -import com.google.common.base.Preconditions; - -public class TokenProcessor extends SimpleProcessor { - IntWritable one = new IntWritable(1); - Text word = new Text(); - static String INPUT = "Input"; - static String SUMMATION = "Summation"; - - public TokenProcessor(ProcessorContext context) { - super(context); - } - - @Override - public void run() throws Exception { - Preconditions.checkArgument(getInputs().size() == 1); - Preconditions.checkArgument(getOutputs().size() == 1); - // the recommended approach is to cast the reader/writer to a specific type instead - // of casting the input/output. This allows the actual input/output type to be replaced - // without affecting the semantic guarantees of the data type that are represented by - // the reader and writer. - // The inputs/outputs are referenced via the names assigned in the DAG. - KeyValueReader kvReader = (KeyValueReader) getInputs().get(INPUT).getReader(); - KeyValueWriter kvWriter = (KeyValueWriter) getOutputs().get(SUMMATION).getWriter(); - while (kvReader.next()) { - StringTokenizer itr = new StringTokenizer(kvReader.getCurrentValue().toString()); - while (itr.hasMoreTokens()) { - word.set(itr.nextToken()); - // Count 1 every time a word is observed. Word is the key a 1 is the value - kvWriter.write(word, one); - } - } - } - -} diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/FakeInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/FakeInput.java deleted file mode 100644 index 3e2d644d72..0000000000 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/FakeInput.java +++ /dev/null @@ -1,80 +0,0 @@ -package org.apache.tez.runtime.library; - -import java.io.IOException; -import java.util.List; - -import org.apache.tez.runtime.api.AbstractLogicalInput; -import org.apache.tez.runtime.api.Event; -import org.apache.tez.runtime.api.InputContext; -import org.apache.tez.runtime.api.Reader; -import org.apache.tez.runtime.library.api.KeyValueReader; - -public class FakeInput extends AbstractLogicalInput { - - private static final int numRecordPerSrc = 10; - - /** - * Constructor an instance of the LogicalInput. Classes extending this one to create a - * LogicalInput, must provide the same constructor so that Tez can create an instance of the - * class at runtime. - * - * @param inputContext the {@link InputContext} which provides - * the Input with context information within the running task. - * @param numPhysicalInputs the number of physical inputs that the logical input will - */ - public FakeInput(InputContext inputContext, int numPhysicalInputs) { - super(inputContext, numPhysicalInputs); - } - - @Override - public List initialize() throws Exception { - getContext().requestInitialMemory(0, null); - getContext().inputIsReady(); - return null; - } - - @Override - public void handleEvents(List inputEvents) throws Exception { - - } - - @Override - public List close() throws Exception { - return null; - } - - @Override - public void start() throws Exception { - - } - - @Override - public Reader getReader() throws Exception { - return new KeyValueReader() { - String[] keys = new String[numRecordPerSrc]; - - int i = -1; - - @Override - public boolean next() throws IOException { - if (i == -1) { - for (int j = 0; j < numRecordPerSrc; j++) { - keys[j] = ""+j; - } - } - i++; - return i < keys.length; - } - - @Override - public Object getCurrentKey() throws IOException { - return keys[i]; - } - - @Override - public Object getCurrentValue() throws IOException { - return keys[i]; - } - }; - } -} diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/FakeInputInitializer.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/FakeInputInitializer.java deleted file mode 100644 index fb064fbe47..0000000000 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/FakeInputInitializer.java +++ /dev/null @@ -1,43 +0,0 @@ -package org.apache.tez.runtime.library; - -import java.util.ArrayList; -import java.util.List; - -import org.apache.tez.runtime.api.Event; -import org.apache.tez.runtime.api.InputInitializer; -import org.apache.tez.runtime.api.InputInitializerContext; -import org.apache.tez.runtime.api.events.InputConfigureVertexTasksEvent; -import org.apache.tez.runtime.api.events.InputDataInformationEvent; -import org.apache.tez.runtime.api.events.InputInitializerEvent; - -public class FakeInputInitializer extends InputInitializer { - - private static final int srcParallelism = 1; - - /** - * Constructor an instance of the InputInitializer. Classes extending this to create a - * InputInitializer, must provide the same constructor so that Tez can create an instance of - * the class at runtime. - * - * @param initializerContext initializer context which can be used to access the payload, vertex - * properties, etc - */ - public FakeInputInitializer(InputInitializerContext initializerContext) { - super(initializerContext); - } - - @Override - public List initialize() throws Exception { - List list = new ArrayList<>(); - list.add(InputConfigureVertexTasksEvent.create(srcParallelism, null, null)); - for (int i = 0; i < srcParallelism; i++) { - list.add(InputDataInformationEvent.createWithObjectPayload(i, null)); - } - return list; - } - - @Override - public void handleInputInitializerEvent(List events) throws Exception { - - } -} diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/FakeOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/FakeOutput.java deleted file mode 100644 index e5ef6e5fae..0000000000 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/FakeOutput.java +++ /dev/null @@ -1,58 +0,0 @@ -package org.apache.tez.runtime.library; - -import java.io.IOException; -import java.util.List; - -import org.apache.tez.runtime.api.AbstractLogicalOutput; -import org.apache.tez.runtime.api.Event; -import org.apache.tez.runtime.api.OutputContext; -import org.apache.tez.runtime.api.Writer; -import org.apache.tez.runtime.library.api.KeyValueWriter; - -public class FakeOutput extends AbstractLogicalOutput { - - /** - * Constructor an instance of the LogicalOutput. Classes extending this one to create a - * LogicalOutput, must provide the same constructor so that Tez can create an instance of the - * class at runtime. - * - * @param outputContext the {@link OutputContext} which - * provides - * the Output with context information within the running task. - * @param numPhysicalOutputs the number of physical outputs that the logical output will - */ - public FakeOutput(OutputContext outputContext, int numPhysicalOutputs) { - super(outputContext, numPhysicalOutputs); - } - - @Override - public List initialize() throws Exception { - getContext().requestInitialMemory(0, null); - return null; - } - - @Override - public void handleEvents(List outputEvents) { - - } - - @Override - public List close() throws Exception { - return null; - } - - @Override - public void start() throws Exception { - - } - - @Override - public Writer getWriter() throws Exception { - return new KeyValueWriter() { - @Override - public void write(Object key, Object value) throws IOException { - System.out.println(key + " XXX " + value); - } - }; - } -} diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/FakeOutputCommitter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/FakeOutputCommitter.java deleted file mode 100644 index 132335d74b..0000000000 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/FakeOutputCommitter.java +++ /dev/null @@ -1,40 +0,0 @@ -package org.apache.tez.runtime.library; - -import org.apache.tez.dag.api.client.VertexStatus; -import org.apache.tez.runtime.api.OutputCommitter; -import org.apache.tez.runtime.api.OutputCommitterContext; - -public class FakeOutputCommitter extends OutputCommitter { - - /** - * Constructor an instance of the OutputCommitter. Classes extending this to create a - * OutputCommitter, must provide the same constructor so that Tez can create an instance of - * the class at runtime. - * - * @param committerContext committer context which can be used to access the payload, vertex - * properties, etc - */ - public FakeOutputCommitter(OutputCommitterContext committerContext) { - super(committerContext); - } - - @Override - public void initialize() throws Exception { - - } - - @Override - public void setupOutput() throws Exception { - - } - - @Override - public void commitOutput() throws Exception { - - } - - @Override - public void abortOutput(VertexStatus.State finalState) throws Exception { - - } -} diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/package-info.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/package-info.java new file mode 100644 index 0000000000..4fb471f82a --- /dev/null +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/package-info.java @@ -0,0 +1,24 @@ +/** + * 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. + */ + +@Public +@Evolving +package org.apache.tez.runtime.library; + +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Evolving; From b3ca9d5af65a6bc9b5fc51456cf748ecd2f6b1de Mon Sep 17 00:00:00 2001 From: Laszlo Bodor Date: Fri, 21 Nov 2025 08:15:25 +0100 Subject: [PATCH 3/5] checksytle, spotbugs --- tez-api/findbugs-exclude.xml | 8 ++++++++ .../org/apache/tez/client/registry/AMRecord.java | 16 +++++++++------- .../apache/tez/client/registry/AMRegistry.java | 3 ++- .../tez/client/registry/zookeeper/ZkConfig.java | 2 +- .../zookeeper/TestZkFrameworkClient.java | 3 ++- 5 files changed, 22 insertions(+), 10 deletions(-) diff --git a/tez-api/findbugs-exclude.xml b/tez-api/findbugs-exclude.xml index 4952d18c4e..8ce8e6fab8 100644 --- a/tez-api/findbugs-exclude.xml +++ b/tez-api/findbugs-exclude.xml @@ -164,4 +164,12 @@ + + + + + + diff --git a/tez-api/src/main/java/org/apache/tez/client/registry/AMRecord.java b/tez-api/src/main/java/org/apache/tez/client/registry/AMRecord.java index c1c2608883..0a65d9f742 100644 --- a/tez-api/src/main/java/org/apache/tez/client/registry/AMRecord.java +++ b/tez-api/src/main/java/org/apache/tez/client/registry/AMRecord.java @@ -59,14 +59,16 @@ public class AMRecord { * Although this constructor may not be used directly within Tez internals, * it is part of the public API for Tez clients that handle unmanaged sessions. * - * @param appId the {@link ApplicationId} of the Tez application - * @param hostName the hostname where the Application Master is running - * @param hostIp the IP address of the Application Master host - * @param port the RPC port number on which the Application Master is listening - * @param externalId an optional external identifier for the record; if {@code null}, defaults to an empty string - * @param computeName the compute group or cluster name; if {@code null}, defaults to {@link ZkConfig#DEFAULT_COMPUTE_GROUP_NAME} + * @param appId the {@link ApplicationId} of the Tez application + * @param hostName the hostname where the Application Master is running + * @param hostIp the IP address of the Application Master host + * @param port the RPC port number on which the Application Master is listening + * @param externalId an optional external identifier for the record; if {@code null}, defaults to an empty string + * @param computeName the compute group or cluster name; if {@code null}, + * defaults to {@link ZkConfig#DEFAULT_COMPUTE_GROUP_NAME} */ - public AMRecord(ApplicationId appId, String hostName, String hostIp, int port, String externalId, String computeName) { + public AMRecord(ApplicationId appId, String hostName, String hostIp, int port, String externalId, + String computeName) { this.appId = appId; this.hostName = hostName; this.hostIp = hostIp; diff --git a/tez-api/src/main/java/org/apache/tez/client/registry/AMRegistry.java b/tez-api/src/main/java/org/apache/tez/client/registry/AMRegistry.java index e5f96b1f80..c447fe68c8 100644 --- a/tez-api/src/main/java/org/apache/tez/client/registry/AMRegistry.java +++ b/tez-api/src/main/java/org/apache/tez/client/registry/AMRegistry.java @@ -91,7 +91,8 @@ public ApplicationId generateNewId() throws Exception { return null; } - public abstract AMRecord createAmRecord(ApplicationId appId, String hostName, String hostIp, int port, String computeName); + public abstract AMRecord createAmRecord(ApplicationId appId, String hostName, String hostIp, int port, + String computeName); @Override public void serviceStop() throws Exception { diff --git a/tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/ZkConfig.java b/tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/ZkConfig.java index 312c4b1453..fbe131627c 100644 --- a/tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/ZkConfig.java +++ b/tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/ZkConfig.java @@ -63,7 +63,7 @@ public ZkConfig(Configuration conf) { } boolean enableComputeGroups = conf.getBoolean(TezConfiguration.TEZ_AM_REGISTRY_ENABLE_COMPUTE_GROUPS, - TezConfiguration.TEZ_AM_REGISTRY_ENABLE_COMPUTE_GROUPS_DEFAULT); + TezConfiguration.TEZ_AM_REGISTRY_ENABLE_COMPUTE_GROUPS_DEFAULT); if (enableComputeGroups) { final String subNamespace = System.getenv(COMPUTE_GROUP_NAME_ENV); if (subNamespace != null && !subNamespace.isEmpty()) { diff --git a/tez-api/src/test/java/org/apache/tez/client/registry/zookeeper/TestZkFrameworkClient.java b/tez-api/src/test/java/org/apache/tez/client/registry/zookeeper/TestZkFrameworkClient.java index 9a2ce0a033..ab4b3cb7d5 100644 --- a/tez-api/src/test/java/org/apache/tez/client/registry/zookeeper/TestZkFrameworkClient.java +++ b/tez-api/src/test/java/org/apache/tez/client/registry/zookeeper/TestZkFrameworkClient.java @@ -214,7 +214,8 @@ private TezConfiguration createTezConf() { return tezConf; } - private void registerMockAM(TezConfiguration tezConf, ApplicationId appId, String hostName, String hostIp, int port) throws Exception { + private void registerMockAM(TezConfiguration tezConf, ApplicationId appId, String hostName, String hostIp, int port) + throws Exception { // Create AM record and publish it directly to ZooKeeper AMRecord amRecord = new AMRecord(appId, hostName, hostIp, port, "test-external-id", "test-compute"); ServiceRecord serviceRecord = amRecord.toServiceRecord(); From 6bf4f1f242a6d46938828b76c798d6a38ca8b842 Mon Sep 17 00:00:00 2001 From: Laszlo Bodor Date: Fri, 21 Nov 2025 13:57:23 +0100 Subject: [PATCH 4/5] PR comments --- tez-api/findbugs-exclude.xml | 12 + .../apache/tez/client/FrameworkClient.java | 6 +- .../apache/tez/client/registry/AMRecord.java | 35 ++- .../tez/client/registry/AMRegistry.java | 53 +--- .../tez/client/registry/AMRegistryClient.java | 44 ++- .../registry/AMRegistryClientListener.java | 23 +- .../tez/client/registry/AMRegistryUtils.java | 23 +- .../zookeeper/ZkAMRegistryClient.java | 62 +++-- .../client/registry/zookeeper/ZkConfig.java | 54 ++-- .../registry/zookeeper/ZkFrameworkClient.java | 20 +- .../apache/tez/dag/api/TezConfiguration.java | 18 +- .../tez/frameworkplugins/FrameworkUtils.java | 54 ++-- .../org/apache/tez/client/TestTezClient.java | 3 + .../tez/client/registry/TestAMRecord.java | 259 ++++++++++++++++++ .../registry/zookeeper/TestZkConfig.java | 230 ++++++++++++++++ .../zookeeper/TestZkFrameworkClient.java | 29 +- tez-dag/pom.xml | 1 + .../registry/zookeeper/ZkAMRegistry.java | 58 ++-- .../org/apache/tez/dag/app/DAGAppMaster.java | 3 +- .../apache/tez/dag/app/dag/impl/DAGImpl.java | 3 +- .../tez/frameworkplugins/AMExtensions.java | 4 +- .../yarn/YarnServerFrameworkService.java | 18 +- .../zookeeper/ZkStandaloneAMExtensions.java | 2 +- .../ZkStandaloneServerFrameworkService.java | 31 ++- .../zookeeper/TestZkAMRegistryClient.java | 16 +- .../registry/zookeeper/TestZkAMRegistry.java | 192 +++++++++++++ 26 files changed, 991 insertions(+), 262 deletions(-) rename tez-dag/src/main/java/org/apache/tez/frameworkplugins/TaskResourceException.java => tez-api/src/main/java/org/apache/tez/client/registry/AMRegistryUtils.java (51%) create mode 100644 tez-api/src/test/java/org/apache/tez/client/registry/TestAMRecord.java create mode 100644 tez-api/src/test/java/org/apache/tez/client/registry/zookeeper/TestZkConfig.java create mode 100644 tez-dag/src/test/java/org/apache/tez/dag/api/client/registry/zookeeper/TestZkAMRegistry.java diff --git a/tez-api/findbugs-exclude.xml b/tez-api/findbugs-exclude.xml index 8ce8e6fab8..25d41cd95b 100644 --- a/tez-api/findbugs-exclude.xml +++ b/tez-api/findbugs-exclude.xml @@ -172,4 +172,16 @@ returns="void"/> + + + + + + + + + + + + diff --git a/tez-api/src/main/java/org/apache/tez/client/FrameworkClient.java b/tez-api/src/main/java/org/apache/tez/client/FrameworkClient.java index f36a499314..dac20edc08 100644 --- a/tez-api/src/main/java/org/apache/tez/client/FrameworkClient.java +++ b/tez-api/src/main/java/org/apache/tez/client/FrameworkClient.java @@ -67,9 +67,9 @@ public static FrameworkClient createFrameworkClient(TezConfiguration tezConf) { throw new TezUncheckedException("Fail to create LocalClient", e); } } else { - ClientFrameworkService clientFrameworkService = FrameworkUtils.get(ClientFrameworkService.class, tezConf); - return clientFrameworkService == null ? new YarnClientFrameworkService().newFrameworkClient() - : clientFrameworkService.newFrameworkClient(); + ClientFrameworkService clientFrameworkService = FrameworkUtils.get(ClientFrameworkService.class, tezConf, + YarnClientFrameworkService.class); + return clientFrameworkService.newFrameworkClient(); } } diff --git a/tez-api/src/main/java/org/apache/tez/client/registry/AMRecord.java b/tez-api/src/main/java/org/apache/tez/client/registry/AMRecord.java index 0a65d9f742..4fadbf9ed7 100644 --- a/tez-api/src/main/java/org/apache/tez/client/registry/AMRecord.java +++ b/tez-api/src/main/java/org/apache/tez/client/registry/AMRecord.java @@ -19,6 +19,7 @@ package org.apache.tez.client.registry; import java.util.Objects; +import java.util.Optional; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.registry.client.types.ServiceRecord; @@ -51,6 +52,8 @@ public class AMRecord { private final String externalId; private final String computeName; + private ServiceRecord serviceRecord; + /** * Creates a new {@code AMRecord} with the given application ID, host, port, and identifier. *

    @@ -74,8 +77,8 @@ public AMRecord(ApplicationId appId, String hostName, String hostIp, int port, S this.hostIp = hostIp; this.port = port; //externalId is optional, if not provided, convert to empty string - this.externalId = (externalId == null) ? "" : externalId; - this.computeName = (computeName == null) ? ZkConfig.DEFAULT_COMPUTE_GROUP_NAME : computeName; + this.externalId = Optional.ofNullable(externalId).orElse(""); + this.computeName = Optional.ofNullable(computeName).orElse(ZkConfig.DEFAULT_COMPUTE_GROUP_NAME); } /** @@ -89,12 +92,15 @@ public AMRecord(ApplicationId appId, String hostName, String hostIp, int port, S * @param other the {@code AMRecord} instance to copy */ public AMRecord(AMRecord other) { - this.appId = other.getApplicationId(); - this.hostName = other.getHost(); - this.hostIp = other.getHostIp(); - this.port = other.getPort(); - this.externalId = other.getExternalId(); - this.computeName = other.getComputeName(); + this.appId = other.appId; + this.hostName = other.hostName; + this.hostIp = other.hostIp; + this.port = other.port; + this.externalId = other.externalId; + this.computeName = other.computeName; + // all fields are final immutable, we can copy the serviceRecord, + // if it's initialized there already, as it won't change + this.serviceRecord = other.serviceRecord; } /** @@ -121,10 +127,6 @@ public ApplicationId getApplicationId() { return appId; } - public String getHost() { - return hostName; - } - public String getHostName() { return hostName; } @@ -150,8 +152,7 @@ public boolean equals(Object other) { if (this == other) { return true; } - if (other instanceof AMRecord) { - AMRecord otherRecord = (AMRecord) other; + if (other instanceof AMRecord otherRecord) { return appId.equals(otherRecord.appId) && hostName.equals(otherRecord.hostName) && hostIp.equals(otherRecord.hostIp) @@ -178,13 +179,17 @@ public boolean equals(Object other) { * @return a {@link ServiceRecord} populated with the values of this {@code AMRecord} */ public ServiceRecord toServiceRecord() { - ServiceRecord serviceRecord = new ServiceRecord(); + if (serviceRecord != null) { + return serviceRecord; + } + serviceRecord = new ServiceRecord(); serviceRecord.set(APP_ID_RECORD_KEY, appId); serviceRecord.set(HOST_NAME_RECORD_KEY, hostName); serviceRecord.set(HOST_IP_RECORD_KEY, hostIp); serviceRecord.set(PORT_RECORD_KEY, port); serviceRecord.set(EXTERNAL_ID_KEY, externalId); serviceRecord.set(COMPUTE_GROUP_NAME_KEY, computeName); + return serviceRecord; } diff --git a/tez-api/src/main/java/org/apache/tez/client/registry/AMRegistry.java b/tez-api/src/main/java/org/apache/tez/client/registry/AMRegistry.java index c447fe68c8..bf87c5e9ec 100644 --- a/tez-api/src/main/java/org/apache/tez/client/registry/AMRegistry.java +++ b/tez-api/src/main/java/org/apache/tez/client/registry/AMRegistry.java @@ -18,16 +18,10 @@ package org.apache.tez.client.registry; -import java.util.ArrayList; -import java.util.List; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.service.AbstractService; -import org.apache.hadoop.service.ServiceStateException; import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Base class for {@code AMRegistry} implementations. @@ -43,62 +37,19 @@ *

  • {@code serviceStop}
  • *
*

- * - *

{@code init} and {@code serviceStart} are invoked during - * {@code DAGAppMaster.serviceInit()}, while {@code serviceStop} is called - * when {@code DAGAppMaster} shuts down.

*/ public abstract class AMRegistry extends AbstractService { - - private static final Logger LOG = LoggerFactory.getLogger(AMRegistry.class); - private List amRecords = new ArrayList<>(); - - @Override - public void init(Configuration conf) { - try { - this.serviceInit(conf); - } catch (Exception e) { - LOG.error("Failed to init AMRegistry: name={}, type={}", getName(), getClass().getName()); - throw ServiceStateException.convert(e); - } - } - - @Override - public void start() { - try { - this.serviceStart(); - } catch(Exception e) { - LOG.error("Failed to start AMRegistry: name={}, type={}", getName(), getClass().getName()); - throw ServiceStateException.convert(e); - } - } - /* Implementations should provide a public no-arg constructor */ protected AMRegistry(String name) { super(name); } - /* Under typical usage, add will be called once automatically with an AMRecord - for the DAGClientServer servicing an AM - */ - public void add(AMRecord server) throws Exception { - amRecords.add(server); - } + public abstract void add(AMRecord server) throws Exception; public abstract void remove(AMRecord server) throws Exception; - public ApplicationId generateNewId() throws Exception { - return null; - } + public abstract ApplicationId generateNewId() throws Exception; public abstract AMRecord createAmRecord(ApplicationId appId, String hostName, String hostIp, int port, String computeName); - - @Override - public void serviceStop() throws Exception { - List records = new ArrayList<>(amRecords); - for(AMRecord record : records) { - remove(record); - } - } } diff --git a/tez-api/src/main/java/org/apache/tez/client/registry/AMRegistryClient.java b/tez-api/src/main/java/org/apache/tez/client/registry/AMRegistryClient.java index e3cd1dfe04..02e320fc1c 100644 --- a/tez-api/src/main/java/org/apache/tez/client/registry/AMRegistryClient.java +++ b/tez-api/src/main/java/org/apache/tez/client/registry/AMRegistryClient.java @@ -23,6 +23,11 @@ import java.util.ArrayList; import java.util.List; +import org.apache.hadoop.yarn.api.records.ApplicationId; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + /** * Client-side interface for discovering Application Master (AM) instances * registered in the AM registry. @@ -35,18 +40,10 @@ * appear or are removed.

*/ public abstract class AMRegistryClient implements Closeable { + private static final Logger LOG = LoggerFactory.getLogger(AMRegistryClient.class); private final List listeners = new ArrayList<>(); - /** - * Returns the current set of registered listeners. - * - * @return a mutable list of listeners - */ - protected List getListeners() { - return listeners; - } - /** * Lookup AM metadata for the given application ID. * @@ -54,7 +51,7 @@ protected List getListeners() { * @return the AM record if found, otherwise {@code null} * @throws IOException if the lookup fails */ - public abstract AMRecord getRecord(String appId) throws IOException; + public abstract AMRecord getRecord(ApplicationId appId) throws IOException; /** * Retrieve all AM records known in the registry. @@ -81,7 +78,26 @@ public synchronized void addListener(AMRegistryClientListener listener) { */ protected synchronized void notifyOnAdded(AMRecord record) { for (AMRegistryClientListener listener : listeners) { - listener.onAdd(record); + try { + listener.onAdd(record); + } catch (Exception e) { + LOG.warn("Exception while calling AM add listener, AM record {}", record, e); + } + } + } + + /** + * Notify listeners of an updated AM record. + * + * @param record the updated AM record + */ + protected synchronized void notifyOnUpdated(AMRecord record) { + for (AMRegistryClientListener listener : listeners) { + try { + listener.onUpdate(record); + } catch (Exception e) { + LOG.warn("Exception while calling AM update listener, AM record {}", record, e); + } } } @@ -92,7 +108,11 @@ protected synchronized void notifyOnAdded(AMRecord record) { */ protected synchronized void notifyOnRemoved(AMRecord record) { for (AMRegistryClientListener listener : listeners) { - listener.onRemove(record); + try { + listener.onRemove(record); + } catch (Exception e) { + LOG.warn("Exception while calling AM remove listener, AM record {}", record, e); + } } } } diff --git a/tez-api/src/main/java/org/apache/tez/client/registry/AMRegistryClientListener.java b/tez-api/src/main/java/org/apache/tez/client/registry/AMRegistryClientListener.java index 527d821bde..789fc22ef2 100644 --- a/tez-api/src/main/java/org/apache/tez/client/registry/AMRegistryClientListener.java +++ b/tez-api/src/main/java/org/apache/tez/client/registry/AMRegistryClientListener.java @@ -6,9 +6,9 @@ * 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 - * + *

+ * 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. @@ -19,6 +19,23 @@ package org.apache.tez.client.registry; public interface AMRegistryClientListener { + void onAdd(AMRecord record); + + /** + * Default implementation of {@code onUpdate} delegates to {@code onAdd}. + * + *

This provides a convenient backward-compatible behavior for consumers that + * store {@link AMRecord} instances in collections keyed by something stable + * (such as ApplicationId). In such cases, re-adding an {@link AMRecord} + * effectively overwrites the previous entry, making an explicit update handler + * unnecessary for many implementations.

+ * + * @param record the updated {@link AMRecord} instance + */ + default void onUpdate(AMRecord record){ + onAdd(record); + } + void onRemove(AMRecord record); } diff --git a/tez-dag/src/main/java/org/apache/tez/frameworkplugins/TaskResourceException.java b/tez-api/src/main/java/org/apache/tez/client/registry/AMRegistryUtils.java similarity index 51% rename from tez-dag/src/main/java/org/apache/tez/frameworkplugins/TaskResourceException.java rename to tez-api/src/main/java/org/apache/tez/client/registry/AMRegistryUtils.java index 697c2a7697..79b372c683 100644 --- a/tez-dag/src/main/java/org/apache/tez/frameworkplugins/TaskResourceException.java +++ b/tez-api/src/main/java/org/apache/tez/client/registry/AMRegistryUtils.java @@ -15,10 +15,25 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.tez.frameworkplugins; +package org.apache.tez.client.registry; -public class TaskResourceException extends Exception { - public TaskResourceException(String msg) { - super(msg); +import java.io.IOException; + +import org.apache.hadoop.registry.client.binding.RegistryUtils; +import org.apache.hadoop.registry.client.types.ServiceRecord; + +public final class AMRegistryUtils { + + private AMRegistryUtils() {} + + public static AMRecord jsonStringToRecord(String json) throws IOException { + RegistryUtils.ServiceRecordMarshal marshal = new RegistryUtils.ServiceRecordMarshal(); + ServiceRecord serviceRecord = marshal.fromJson(json); + return new AMRecord(serviceRecord); + } + + public static String recordToJsonString(AMRecord amRecord) throws IOException { + RegistryUtils.ServiceRecordMarshal marshal = new RegistryUtils.ServiceRecordMarshal(); + return marshal.toJson(amRecord.toServiceRecord()); } } diff --git a/tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/ZkAMRegistryClient.java b/tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/ZkAMRegistryClient.java index dc7d9334dd..7e189490da 100644 --- a/tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/ZkAMRegistryClient.java +++ b/tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/ZkAMRegistryClient.java @@ -19,27 +19,27 @@ package org.apache.tez.client.registry.zookeeper; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.stream.Collectors; +import org.apache.commons.io.IOUtils; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.imps.CuratorFrameworkState; import org.apache.curator.framework.recipes.cache.ChildData; import org.apache.curator.framework.recipes.cache.TreeCache; import org.apache.curator.framework.recipes.cache.TreeCacheEvent; import org.apache.curator.framework.recipes.cache.TreeCacheListener; -import org.apache.curator.shaded.com.google.common.base.Charsets; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.registry.client.binding.RegistryUtils; import org.apache.hadoop.registry.client.types.ServiceRecord; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.tez.client.registry.AMRecord; import org.apache.tez.client.registry.AMRegistryClient; -import org.apache.tez.client.registry.AMRegistryClientListener; +import org.apache.tez.client.registry.AMRegistryUtils; import org.apache.tez.dag.api.TezConfiguration; import com.fasterxml.jackson.core.JsonParseException; @@ -57,9 +57,12 @@ public final class ZkAMRegistryClient extends AMRegistryClient { private static final Map INSTANCES = new HashMap<>(); private final Configuration conf; - //Cache of known AMs - private final ConcurrentHashMap amRecordCache = new ConcurrentHashMap<>(); + // Cache of known AMs + private final ConcurrentHashMap amRecordCache = new ConcurrentHashMap<>(); + private CuratorFramework client; + private TreeCache cache; + private ZkRegistryListener listener; private ZkAMRegistryClient(final Configuration conf) { this.conf = conf; @@ -86,7 +89,7 @@ public static synchronized ZkAMRegistryClient getClient(final Configuration conf * @throws IOException if the data cannot be deserialized into a {@link ServiceRecord} */ public static AMRecord getAMRecord(final ChildData childData) throws IOException { - // not a leaf path. Only leaf path contains AMRecord + // Not a leaf path. Only leaf path contains AMRecord. if (!childData.getPath().contains(ApplicationId.appIdStrPrefix)) { return null; } @@ -95,11 +98,9 @@ public static AMRecord getAMRecord(final ChildData childData) throws IOException if (data.length == 0) { return null; } - String value = new String(data, Charsets.UTF_8); + String value = new String(data, StandardCharsets.UTF_8); try { - RegistryUtils.ServiceRecordMarshal marshal = new RegistryUtils.ServiceRecordMarshal(); - ServiceRecord serviceRecord = marshal.fromJson(value); - return new AMRecord(serviceRecord); + return AMRegistryUtils.jsonStringToRecord(value); } catch (JsonParseException e) { //Not a json AMRecord (SRV), could be some other data. LOG.warn("Non-json data received while de-serializing AMRecord: {}. Ignoring...", value); @@ -110,19 +111,18 @@ public static AMRecord getAMRecord(final ChildData childData) throws IOException public void start() throws Exception { ZkConfig zkConf = new ZkConfig(this.conf); client = zkConf.createCuratorFramework(); - final TreeCache cache = new TreeCache(client, zkConf.getZkNamespace()); + cache = new TreeCache(client, zkConf.getZkNamespace()); client.start(); cache.start(); - cache.getListenable().addListener(new ZkRegistryListener()); + listener = new ZkRegistryListener(); + cache.getListenable().addListener(listener); } @Override - public AMRecord getRecord(String appId) { - if (amRecordCache.get(appId) == null) { - return null; - } - //Return a copy - return new AMRecord(amRecordCache.get(appId)); + public AMRecord getRecord(ApplicationId appId) { + AMRecord rec = amRecordCache.get(appId); + // Return a copy. + return rec == null ? null : new AMRecord(rec); } @Override @@ -131,13 +131,13 @@ public List getAllRecords() { } @Override - public synchronized void addListener(AMRegistryClientListener listener) { - getListeners().add(listener); + public void close() { + IOUtils.closeQuietly(cache); + IOUtils.closeQuietly(client); } - @Override - public void close() { - client.close(); + public boolean isInitialized() { + return listener.initialized; } /** @@ -146,6 +146,8 @@ public void close() { */ private class ZkRegistryListener implements TreeCacheListener { + private boolean initialized = false; + @Override public void childEvent(final CuratorFramework clientParam, final TreeCacheEvent event) throws Exception { Preconditions.checkArgument(clientParam != null && clientParam.getState() == CuratorFrameworkState.STARTED, @@ -159,8 +161,8 @@ public void childEvent(final CuratorFramework clientParam, final TreeCacheEvent } else { AMRecord amRecord = getAMRecord(childData); if (amRecord != null) { - LOG.info("AM registered with data: {}. Notifying {} listeners.", amRecord, getListeners().size()); - amRecordCache.put(amRecord.getApplicationId().toString(), amRecord); + LOG.info("AM registered with data: {}. Notifying listeners.", amRecord); + amRecordCache.put(amRecord.getApplicationId(), amRecord); notifyOnAdded(amRecord); } } @@ -171,8 +173,8 @@ public void childEvent(final CuratorFramework clientParam, final TreeCacheEvent } else { AMRecord amRecord = getAMRecord(childData); if (amRecord != null) { - LOG.info("AM updated data: {}. Notifying {} listeners.", amRecord, getListeners().size()); - amRecordCache.put(amRecord.getApplicationId().toString(), amRecord); + LOG.info("AM updated data: {}. Notifying listeners.", amRecord); + amRecordCache.put(amRecord.getApplicationId(), amRecord); notifyOnAdded(amRecord); } } @@ -183,12 +185,14 @@ public void childEvent(final CuratorFramework clientParam, final TreeCacheEvent } else { AMRecord amRecord = getAMRecord(childData); if (amRecord != null) { - LOG.info("AM removed: {}. Notifying {} listeners.", amRecord, getListeners().size()); - amRecordCache.remove(amRecord.getApplicationId().toString(), amRecord); + LOG.info("AM removed: {}. Notifying listeners.", amRecord); + amRecordCache.remove(amRecord.getApplicationId(), amRecord); notifyOnRemoved(amRecord); } } break; + case INITIALIZED: + this.initialized = true; default: if (childData == null) { LOG.info("Ignored event {}", event.getType()); diff --git a/tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/ZkConfig.java b/tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/ZkConfig.java index fbe131627c..af59072532 100644 --- a/tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/ZkConfig.java +++ b/tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/ZkConfig.java @@ -18,6 +18,8 @@ package org.apache.tez.client.registry.zookeeper; +import java.util.concurrent.TimeUnit; + import org.apache.curator.RetryPolicy; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; @@ -27,6 +29,7 @@ import org.apache.tez.dag.api.TezConfiguration; import com.google.common.base.Preconditions; +import com.google.common.base.Strings; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -42,51 +45,45 @@ public class ZkConfig { private final String zkQuorum; private final String zkNamespace; - private final int curatorBackoffSleep; + private final int curatorBackoffSleepMs; private final int curatorMaxRetries; private final int sessionTimeoutMs; private final int connectionTimeoutMs; public ZkConfig(Configuration conf) { zkQuorum = conf.get(TezConfiguration.TEZ_AM_ZOOKEEPER_QUORUM); - Preconditions.checkNotNull(zkQuorum); + Preconditions.checkArgument(!Strings.isNullOrEmpty(zkQuorum), "zkQuorum cannot be null or empty"); String fullZkNamespace = ZK_NAMESPACE_PREFIX; String namespace = conf.get(TezConfiguration.TEZ_AM_REGISTRY_NAMESPACE, TezConfiguration.TEZ_AM_REGISTRY_NAMESPACE_DEFAULT); - Preconditions.checkNotNull(namespace); - if (namespace.startsWith(Path.SEPARATOR)) { - fullZkNamespace += namespace; - } else { - fullZkNamespace = fullZkNamespace + Path.SEPARATOR + namespace; - } + Preconditions.checkArgument(!Strings.isNullOrEmpty(namespace), "namespace cannot be null or empty"); + + fullZkNamespace = appendNamespace(fullZkNamespace, namespace); boolean enableComputeGroups = conf.getBoolean(TezConfiguration.TEZ_AM_REGISTRY_ENABLE_COMPUTE_GROUPS, TezConfiguration.TEZ_AM_REGISTRY_ENABLE_COMPUTE_GROUPS_DEFAULT); if (enableComputeGroups) { final String subNamespace = System.getenv(COMPUTE_GROUP_NAME_ENV); if (subNamespace != null && !subNamespace.isEmpty()) { - if (subNamespace.startsWith(Path.SEPARATOR)) { - fullZkNamespace += subNamespace; - } else { - fullZkNamespace = fullZkNamespace + Path.SEPARATOR + subNamespace; - } + fullZkNamespace = appendNamespace(fullZkNamespace, subNamespace); LOG.info("Compute groups enabled: subNamespace: {} fullZkNamespace: {}", subNamespace, fullZkNamespace); } } else { LOG.info("Compute groups disabled: fullZkNamespace: {}", fullZkNamespace); } zkNamespace = fullZkNamespace; + LOG.info("Using ZK namespace: {}", fullZkNamespace); - curatorBackoffSleep = conf.getInt(TezConfiguration.TEZ_AM_CURATOR_BACKOFF_SLEEP, - TezConfiguration.TEZ_AM_CURATOR_BACKOFF_SLEEP_DEFAULT); + curatorBackoffSleepMs = Math.toIntExact(conf.getTimeDuration(TezConfiguration.TEZ_AM_CURATOR_BACKOFF_SLEEP, + TezConfiguration.TEZ_AM_CURATOR_BACKOFF_SLEEP_DEFAULT, TimeUnit.MILLISECONDS)); curatorMaxRetries = conf.getInt(TezConfiguration.TEZ_AM_CURATOR_MAX_RETRIES, TezConfiguration.TEZ_AM_CURATOR_MAX_RETRIES_DEFAULT); - sessionTimeoutMs = conf.getInt(TezConfiguration.TEZ_AM_CURATOR_SESSION_TIMEOUT, - TezConfiguration.TEZ_AM_CURATOR_SESSION_TIMEOUT_DEFAULT); - connectionTimeoutMs = conf.getInt(TezConfiguration.TEZ_AM_CURATOR_CONNECTION_TIMEOUT, - TezConfiguration.TEZ_AM_CURATOR_CONNECTION_TIMEOUT_DEFAULT); + sessionTimeoutMs = Math.toIntExact(conf.getTimeDuration(TezConfiguration.TEZ_AM_CURATOR_SESSION_TIMEOUT, + TezConfiguration.TEZ_AM_CURATOR_SESSION_TIMEOUT_DEFAULT, TimeUnit.MILLISECONDS)); + connectionTimeoutMs = Math.toIntExact(conf.getTimeDuration(TezConfiguration.TEZ_AM_CURATOR_CONNECTION_TIMEOUT, + TezConfiguration.TEZ_AM_CURATOR_CONNECTION_TIMEOUT_DEFAULT, TimeUnit.MILLISECONDS)); } public String getZkQuorum() { @@ -97,8 +94,8 @@ public String getZkNamespace() { return zkNamespace; } - public int getCuratorBackoffSleep() { - return curatorBackoffSleep; + public int getCuratorBackoffSleepMs() { + return curatorBackoffSleepMs; } public int getCuratorMaxRetries() { @@ -114,7 +111,7 @@ public int getConnectionTimeoutMs() { } public RetryPolicy getRetryPolicy() { - return new ExponentialBackoffRetry(getCuratorBackoffSleep(), getCuratorMaxRetries()); + return new ExponentialBackoffRetry(getCuratorBackoffSleepMs(), getCuratorMaxRetries()); } public CuratorFramework createCuratorFramework() { @@ -125,4 +122,17 @@ public CuratorFramework createCuratorFramework() { getRetryPolicy() ); } + + /** + * Appends a namespace to the given prefix, inserting a path separator between + * them if necessary. + * + * @param prefix the initial path prefix to which the namespace is appended; must not be null + * @param namespace the namespace segment to append; must not be null + * @return the concatenation of {@code prefix} and {@code namespace} with a separator inserted if needed + */ + private String appendNamespace(String prefix, String namespace) { + boolean hasSlash = namespace.startsWith(Path.SEPARATOR); + return prefix + (hasSlash ? namespace : Path.SEPARATOR + namespace); + } } diff --git a/tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/ZkFrameworkClient.java b/tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/ZkFrameworkClient.java index d743defcb5..ef9f948039 100644 --- a/tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/ZkFrameworkClient.java +++ b/tez-api/src/main/java/org/apache/tez/client/registry/zookeeper/ZkFrameworkClient.java @@ -33,6 +33,8 @@ import org.apache.tez.client.registry.AMRecord; import org.apache.tez.dag.api.TezConfiguration; +import com.google.common.annotations.VisibleForTesting; + public class ZkFrameworkClient extends FrameworkClient { private AMRecord amRecord; @@ -43,10 +45,9 @@ public class ZkFrameworkClient extends FrameworkClient { @Override public synchronized void init(TezConfiguration tezConf) { - if (this.amRegistryClient == null) { + if (amRegistryClient == null) { try { - this.amRegistryClient = ZkAMRegistryClient.getClient(tezConf); - this.isRunning = true; + amRegistryClient = ZkAMRegistryClient.getClient(tezConf); } catch (Exception e) { throw new RuntimeException(e); } @@ -56,7 +57,7 @@ public synchronized void init(TezConfiguration tezConf) { @Override public void start() { try { - this.amRegistryClient.start(); + amRegistryClient.start(); isRunning = true; } catch (Exception e) { throw new RuntimeException(e); @@ -131,15 +132,15 @@ public ApplicationReport getApplicationReport(ApplicationId appId) throws YarnEx ApplicationReport report = Records.newRecord(ApplicationReport.class); report.setApplicationId(appId); report.setTrackingUrl(""); - amRecord = amRegistryClient.getRecord(appId.toString()); + amRecord = amRegistryClient.getRecord(appId); // this could happen if the AM died, the AM record store under path will not exist if (amRecord == null) { report.setYarnApplicationState(YarnApplicationState.FINISHED); report.setFinalApplicationStatus(FinalApplicationStatus.FAILED); report.setDiagnostics("AM record not found (likely died) in zookeeper for application id: " + appId); } else { - report.setHost(amRecord.getHost()); - amHost = amRecord.getHost(); + report.setHost(amRecord.getHostName()); + amHost = amRecord.getHostName(); amPort = amRecord.getPort(); report.setRpcPort(amRecord.getPort()); report.setYarnApplicationState(YarnApplicationState.RUNNING); @@ -161,4 +162,9 @@ public String getAmHost() { public int getAmPort() { return amPort; } + + @VisibleForTesting + boolean isZkInitialized() { + return amRegistryClient.isInitialized(); + } } diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java index 5de35bdd82..6fc61dddbe 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java @@ -2374,13 +2374,13 @@ static Set getPropertySet() { /** - * Integer value. Initial backoff sleep duration (in milliseconds) for Curator retries. - * It's used when creating a CuratorFramework for the ZooKeeper registry. + * Initial backoff sleep duration for Curator retries. Supports TimeUnits. + * Default unit is milliseconds. It's used when creating a CuratorFramework for the ZooKeeper registry. */ @ConfigurationScope(Scope.AM) @ConfigurationProperty public static final String TEZ_AM_CURATOR_BACKOFF_SLEEP = TEZ_AM_PREFIX + "curator.backoff.sleep"; - public static final int TEZ_AM_CURATOR_BACKOFF_SLEEP_DEFAULT = 1000; + public static final String TEZ_AM_CURATOR_BACKOFF_SLEEP_DEFAULT = "1000ms"; /** * Integer value. Maximum number of retries for Curator operations. @@ -2392,22 +2392,22 @@ static Set getPropertySet() { public static final int TEZ_AM_CURATOR_MAX_RETRIES_DEFAULT = 3; /** - * Integer value. Session timeout (in milliseconds) for Curator framework. - * It's used when creating a CuratorFramework for the ZooKeeper registry. + * Session timeout for Curator framework. Supports TimeUnits. + * Default unit is milliseconds. It's used when creating a CuratorFramework for the ZooKeeper registry. */ @ConfigurationScope(Scope.AM) @ConfigurationProperty public static final String TEZ_AM_CURATOR_SESSION_TIMEOUT = TEZ_AM_PREFIX + "curator.session.timeout"; - public static final int TEZ_AM_CURATOR_SESSION_TIMEOUT_DEFAULT = 150000; + public static final String TEZ_AM_CURATOR_SESSION_TIMEOUT_DEFAULT = "150000ms"; /** - * Integer value. Connection timeout (in milliseconds) for Curator framework. - * It's used when creating a CuratorFramework for the ZooKeeper registry. + * Connection timeout for Curator framework. Supports TimeUnits. + * Default unit is milliseconds. It's used when creating a CuratorFramework for the ZooKeeper registry. */ @ConfigurationScope(Scope.AM) @ConfigurationProperty public static final String TEZ_AM_CURATOR_CONNECTION_TIMEOUT = TEZ_AM_PREFIX + "curator.connection.timeout"; - public static final int TEZ_AM_CURATOR_CONNECTION_TIMEOUT_DEFAULT = 15000; + public static final String TEZ_AM_CURATOR_CONNECTION_TIMEOUT_DEFAULT = "15000ms"; @ConfigurationScope(Scope.AM) diff --git a/tez-api/src/main/java/org/apache/tez/frameworkplugins/FrameworkUtils.java b/tez-api/src/main/java/org/apache/tez/frameworkplugins/FrameworkUtils.java index 51fefd6bf7..9011906fe4 100644 --- a/tez-api/src/main/java/org/apache/tez/frameworkplugins/FrameworkUtils.java +++ b/tez-api/src/main/java/org/apache/tez/frameworkplugins/FrameworkUtils.java @@ -17,9 +17,6 @@ */ package org.apache.tez.frameworkplugins; -import java.util.ArrayList; -import java.util.List; -import java.util.ServiceLoader; import javax.annotation.Nullable; @@ -31,6 +28,9 @@ public final class FrameworkUtils { + private static final String SERVER_FRAMEWORK_SERVICE_INTERFACE_NAME = + "org.apache.tez.frameworkplugins.ServerFrameworkService"; + private FrameworkUtils() {} /* @@ -47,49 +47,37 @@ private FrameworkUtils() {} 2. If conf is null or the parameter TEZ_FRAMEWORK_MODE is not set and the environment var TEZ_FRAMEWORK_MODE is not empty: the value of the environment var will be used - 3. Otherwise: - the default java.util.ServiceLoader behavior will be used, - i.e. the implementation classname should appear in a file on the classpath at the location - META-INF/services/org.apache.tez.frameworkplugins.ClientFrameworkService - or META-INF/services/org.apache.tez.frameworkplugins.ServerFrameworkService + 3. Otherwise: the default class will be instantiated and returned */ - public static T get(Class interfaze, @Nullable Configuration conf) { + public static T get(Class interfaze, @Nullable Configuration conf, + Class defaultClazz) { + String modeInConf = conf != null ? conf.get(TezConfiguration.TEZ_FRAMEWORK_MODE) : null; + String modeInEnv = System.getenv(TezConstants.TEZ_FRAMEWORK_MODE); try { - if ((conf != null) && (conf.get(TezConfiguration.TEZ_FRAMEWORK_MODE) != null)) { - return getByMode(interfaze, conf.get(TezConfiguration.TEZ_FRAMEWORK_MODE)); - } else if (System.getenv(TezConstants.TEZ_FRAMEWORK_MODE) != null) { - return getByMode(interfaze, System.getenv(TezConstants.TEZ_FRAMEWORK_MODE)); + if (modeInConf != null) { + return getByMode(interfaze, modeInConf); + } else if (modeInEnv != null) { + return getByMode(interfaze, modeInEnv); + } else if (defaultClazz != null) { + return (T) defaultClazz.newInstance(); } else { - return getByServiceLoader(interfaze); + throw new RuntimeException( + "Framework service not found in any mode: configuration, environment, or default class"); } - } catch (TezReflectionException e) { + } catch (TezReflectionException | InstantiationException | IllegalAccessException e) { throw new RuntimeException("Failed to load framework service for interface: " + interfaze.getName(), e); } } - private static T getByServiceLoader(Class interfaze) { - List services = new ArrayList<>(); - ServiceLoader frameworkService = ServiceLoader.load(interfaze); - for (T service : frameworkService) { - services.add(service); - } - if (services.isEmpty()) { - return null; - } else if (services.size() > 1) { - throw new RuntimeException("Layering of multiple framework services is not supported." - + " Please provide only one implementation class in configuration."); - } - //services is guaranteed to have one element at this point - return services.getFirst(); - } - private static T getByMode(Class interfaze, String mode) throws TezReflectionException { mode = mode.toUpperCase(); String clazz; - if (interfaze == ClientFrameworkService.class) { + if (ClientFrameworkService.class == interfaze) { clazz = FrameworkMode.valueOf(mode).getClientClassName(); - } else { + } else if (SERVER_FRAMEWORK_SERVICE_INTERFACE_NAME.equals(interfaze.getCanonicalName())) { clazz = FrameworkMode.valueOf(mode).getServerClassName(); + } else { + throw new IllegalArgumentException("Unsupported FrameworkService: " + interfaze.getName()); } return ReflectionUtils.createClazzInstance(clazz); } diff --git a/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java b/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java index fb69241a4d..9d004e8303 100644 --- a/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java +++ b/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java @@ -537,12 +537,15 @@ public void testTezClientReconnect(boolean isSession) throws Exception { Resource.newInstance(1, 1)); dag = DAG.create("DAG2").addVertex(vertex2).addTaskLocalFiles(lrDAG); + dagClient.close(); //DAG 2 submit dagClient = client2.submitDAG(dag); //DAG 2 assertions assertTrue(dagClient.getExecutionContext().contains(appId.toString())); assertEquals(dagClient.getSessionIdentifierString(), appId.toString()); + + dagClient.close(); } @Test (timeout=5000) diff --git a/tez-api/src/test/java/org/apache/tez/client/registry/TestAMRecord.java b/tez-api/src/test/java/org/apache/tez/client/registry/TestAMRecord.java new file mode 100644 index 0000000000..b51b8246e2 --- /dev/null +++ b/tez-api/src/test/java/org/apache/tez/client/registry/TestAMRecord.java @@ -0,0 +1,259 @@ +/** + * 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.tez.client.registry; + +import static org.junit.Assert.*; + +import java.util.concurrent.ConcurrentHashMap; + +import org.apache.hadoop.registry.client.types.ServiceRecord; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.tez.client.registry.zookeeper.ZkConfig; + +import org.junit.Test; + +public class TestAMRecord { + + @Test + public void testConstructorWithAllParameters() { + ApplicationId appId = ApplicationId.newInstance(12345L, 1); + String hostName = "test-host.example.com"; + String hostIp = "192.168.1.100"; + int port = 8080; + String externalId = "external-123"; + String computeName = "test-compute"; + + AMRecord record = new AMRecord(appId, hostName, hostIp, port, externalId, computeName); + + assertEquals(appId, record.getApplicationId()); + assertEquals(hostName, record.getHostName()); + assertEquals(hostName, record.getHostName()); + assertEquals(hostIp, record.getHostIp()); + assertEquals(port, record.getPort()); + assertEquals(externalId, record.getExternalId()); + assertEquals(computeName, record.getComputeName()); + } + + @Test + public void testConstructorWithNullExternalIdAndComputeName() { + ApplicationId appId = ApplicationId.newInstance(12345L, 1); + String hostName = "test-host.example.com"; + String hostIp = "192.168.1.100"; + int port = 8080; + + AMRecord record = new AMRecord(appId, hostName, hostIp, port, null, null); + + assertEquals("", record.getExternalId()); + assertEquals(ZkConfig.DEFAULT_COMPUTE_GROUP_NAME, record.getComputeName()); + } + + @Test + public void testCopyConstructor() { + ApplicationId appId = ApplicationId.newInstance(12345L, 1); + String hostName = "test-host.example.com"; + String hostIp = "192.168.1.100"; + int port = 8080; + String externalId = "external-123"; + String computeName = "test-compute"; + + AMRecord original = new AMRecord(appId, hostName, hostIp, port, externalId, computeName); + AMRecord copy = new AMRecord(original); + + assertEquals(original.getApplicationId(), copy.getApplicationId()); + assertEquals(original.getHostName(), copy.getHostName()); + assertEquals(original.getHostIp(), copy.getHostIp()); + assertEquals(original.getPort(), copy.getPort()); + assertEquals(original.getExternalId(), copy.getExternalId()); + assertEquals(original.getComputeName(), copy.getComputeName()); + assertEquals(original, copy); + assertEquals(original.hashCode(), copy.hashCode()); + } + + @Test + public void testConstructorFromServiceRecord() { + ApplicationId appId = ApplicationId.newInstance(12345L, 1); + String hostName = "test-host.example.com"; + String hostIp = "192.168.1.100"; + int port = 8080; + String externalId = "external-123"; + String computeName = "test-compute"; + + AMRecord original = new AMRecord(appId, hostName, hostIp, port, externalId, computeName); + ServiceRecord serviceRecord = original.toServiceRecord(); + AMRecord reconstructed = new AMRecord(serviceRecord); + + assertEquals(original.getApplicationId(), reconstructed.getApplicationId()); + assertEquals(original.getHostName(), reconstructed.getHostName()); + assertEquals(original.getHostIp(), reconstructed.getHostIp()); + assertEquals(original.getPort(), reconstructed.getPort()); + assertEquals(original.getExternalId(), reconstructed.getExternalId()); + assertEquals(original.getComputeName(), reconstructed.getComputeName()); + assertEquals(original, reconstructed); + } + + @Test + public void testConstructorFromServiceRecordWithNullDefaults() { + ApplicationId appId = ApplicationId.newInstance(12345L, 1); + String hostName = "test-host.example.com"; + String hostIp = "192.168.1.100"; + int port = 8080; + + // Create record with null externalId and computeName + AMRecord original = new AMRecord(appId, hostName, hostIp, port, null, null); + + // Convert to ServiceRecord and back + ServiceRecord serviceRecord = original.toServiceRecord(); + AMRecord reconstructed = new AMRecord(serviceRecord); + + // Verify defaults are preserved + assertEquals("", reconstructed.getExternalId()); + assertEquals(ZkConfig.DEFAULT_COMPUTE_GROUP_NAME, reconstructed.getComputeName()); + assertEquals(original, reconstructed); + } + + @Test + public void testToServiceRecord() { + ApplicationId appId = ApplicationId.newInstance(12345L, 1); + String hostName = "test-host.example.com"; + String hostIp = "192.168.1.100"; + int port = 8080; + String externalId = "external-123"; + String computeName = "test-compute"; + + AMRecord record = new AMRecord(appId, hostName, hostIp, port, externalId, computeName); + ServiceRecord serviceRecord = record.toServiceRecord(); + + assertNotNull(serviceRecord); + assertEquals(appId.toString(), serviceRecord.get("appId")); + assertEquals(hostName, serviceRecord.get("hostName")); + assertEquals(hostIp, serviceRecord.get("hostIp")); + assertEquals(String.valueOf(port), serviceRecord.get("port")); + assertEquals(externalId, serviceRecord.get("externalId")); + assertEquals(computeName, serviceRecord.get("computeName")); + } + + @Test + public void testToServiceRecordCaching() { + ApplicationId appId = ApplicationId.newInstance(12345L, 1); + String hostName = "test-host.example.com"; + String hostIp = "192.168.1.100"; + int port = 8080; + + AMRecord record = new AMRecord(appId, hostName, hostIp, port, "external-123", "test-compute"); + ServiceRecord serviceRecord1 = record.toServiceRecord(); + ServiceRecord serviceRecord2 = record.toServiceRecord(); + + // Should return the same cached instance + assertSame(serviceRecord1, serviceRecord2); + } + + @Test + public void testEquals() { + ApplicationId appId1 = ApplicationId.newInstance(12345L, 1); + ApplicationId appId2 = ApplicationId.newInstance(12345L, 1); + ApplicationId appId3 = ApplicationId.newInstance(12345L, 2); + + AMRecord record1 = new AMRecord(appId1, "host1", "192.168.1.1", 8080, "ext1", "compute1"); + AMRecord record2 = new AMRecord(appId2, "host1", "192.168.1.1", 8080, "ext1", "compute1"); + AMRecord record3 = new AMRecord(appId3, "host1", "192.168.1.1", 8080, "ext1", "compute1"); + AMRecord record4 = new AMRecord(appId1, "host2", "192.168.1.1", 8080, "ext1", "compute1"); + AMRecord record5 = new AMRecord(appId1, "host1", "192.168.1.2", 8080, "ext1", "compute1"); + AMRecord record6 = new AMRecord(appId1, "host1", "192.168.1.1", 8081, "ext1", "compute1"); + AMRecord record7 = new AMRecord(appId1, "host1", "192.168.1.1", 8080, "ext2", "compute1"); + AMRecord record8 = new AMRecord(appId1, "host1", "192.168.1.1", 8080, "ext1", "compute2"); + + // Same values should be equal + assertEquals(record1, record2); + assertEquals(record2, record1); + // Different appId + assertNotEquals(record1, record3); + // Different hostName + assertNotEquals(record1, record4); + // Different hostIp + assertNotEquals(record1, record5); + // Different port + assertNotEquals(record1, record6); + // Different externalId + assertNotEquals(record1, record7); + // Different computeName + assertNotEquals(record1, record8); + // Self equality + assertEquals(record1, record1); + // Null equality + assertNotEquals(null, record1); + // Different type + assertNotEquals("not an AMRecord", record1); + } + + @Test + public void testHashCode() { + ApplicationId appId1 = ApplicationId.newInstance(12345L, 1); + ApplicationId appId2 = ApplicationId.newInstance(12345L, 1); + + AMRecord record1 = new AMRecord(appId1, "host1", "192.168.1.1", 8080, "ext1", "compute1"); + AMRecord record2 = new AMRecord(appId2, "host1", "192.168.1.1", 8080, "ext1", "compute1"); + AMRecord record3 = new AMRecord(appId1, "host2", "192.168.1.1", 8080, "ext1", "compute1"); + + // Equal objects should have same hashCode + assertEquals(record1.hashCode(), record2.hashCode()); + } + + @Test + public void testToString() { + ApplicationId appId = ApplicationId.newInstance(12345L, 1); + String hostName = "test-host.example.com"; + String hostIp = "192.168.1.100"; + int port = 8080; + String externalId = "external-123"; + String computeName = "test-compute"; + + AMRecord record = new AMRecord(appId, hostName, hostIp, port, externalId, computeName); + String str = record.toString(); + + assertNotNull(str); + // Validate actual JSON-like snippets from the string + assertTrue("Should contain appId=value snippet", str.contains("appId=" + appId.toString())); + assertTrue("Should contain hostName=value snippet", str.contains("hostName=" + hostName)); + assertTrue("Should contain hostIp=value snippet", str.contains("hostIp=" + hostIp)); + assertTrue("Should contain port=value snippet", str.contains("port=" + port)); + assertTrue("Should contain externalId=value snippet", str.contains("externalId=" + externalId)); + assertTrue("Should contain computeName=value snippet", str.contains("computeName=" + computeName)); + } + + @Test + public void testRemoveFromCacheByDeserializedRecordAppId() throws Exception { + ConcurrentHashMap amRecordCache = new ConcurrentHashMap<>(); + + ApplicationId appId = ApplicationId.newInstance(12345L, 1); + String hostName = "test-host.example.com"; + String hostIp = "192.168.1.100"; + int port = 8080; + String externalId = "external-123"; + String computeName = "test-compute"; + + AMRecord record = new AMRecord(appId, hostName, hostIp, port, externalId, computeName); + amRecordCache.put(appId, record); + + assertEquals(1, amRecordCache.size()); + + AMRecord deserialized = AMRegistryUtils.jsonStringToRecord(AMRegistryUtils.recordToJsonString(record)); + amRecordCache.remove(deserialized.getApplicationId()); + + assertEquals(0, amRecordCache.size()); + } +} diff --git a/tez-api/src/test/java/org/apache/tez/client/registry/zookeeper/TestZkConfig.java b/tez-api/src/test/java/org/apache/tez/client/registry/zookeeper/TestZkConfig.java new file mode 100644 index 0000000000..7a4ab20db4 --- /dev/null +++ b/tez-api/src/test/java/org/apache/tez/client/registry/zookeeper/TestZkConfig.java @@ -0,0 +1,230 @@ +/** + * 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.tez.client.registry.zookeeper; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + +import java.util.concurrent.TimeUnit; + +import org.apache.curator.RetryPolicy; +import org.apache.curator.framework.CuratorFramework; +import org.apache.hadoop.conf.Configuration; +import org.apache.tez.dag.api.TezConfiguration; + +import org.junit.Test; + +public class TestZkConfig { + + @Test + public void testBasicConfiguration() { + TezConfiguration conf = new TezConfiguration(); + conf.set(TezConfiguration.TEZ_AM_ZOOKEEPER_QUORUM, "localhost:2181"); + conf.set(TezConfiguration.TEZ_AM_REGISTRY_NAMESPACE, "/test-namespace"); + + ZkConfig zkConfig = new ZkConfig(conf); + + assertEquals("localhost:2181", zkConfig.getZkQuorum()); + assertEquals("/tez-external-sessions/test-namespace", zkConfig.getZkNamespace()); + } + + @Test + public void testDefaultValues() { + TezConfiguration conf = new TezConfiguration(); + conf.set(TezConfiguration.TEZ_AM_ZOOKEEPER_QUORUM, "localhost:2181"); + + ZkConfig zkConfig = new ZkConfig(conf); + + Configuration defaultConf = new Configuration(); + long expectedBackoffSleep = defaultConf.getTimeDuration( + TezConfiguration.TEZ_AM_CURATOR_BACKOFF_SLEEP, + TezConfiguration.TEZ_AM_CURATOR_BACKOFF_SLEEP_DEFAULT, TimeUnit.MILLISECONDS); + long expectedSessionTimeout = defaultConf.getTimeDuration( + TezConfiguration.TEZ_AM_CURATOR_SESSION_TIMEOUT, + TezConfiguration.TEZ_AM_CURATOR_SESSION_TIMEOUT_DEFAULT, TimeUnit.MILLISECONDS); + long expectedConnectionTimeout = defaultConf.getTimeDuration( + TezConfiguration.TEZ_AM_CURATOR_CONNECTION_TIMEOUT, + TezConfiguration.TEZ_AM_CURATOR_CONNECTION_TIMEOUT_DEFAULT, TimeUnit.MILLISECONDS); + + assertEquals(expectedBackoffSleep, zkConfig.getCuratorBackoffSleepMs()); + assertEquals(TezConfiguration.TEZ_AM_CURATOR_MAX_RETRIES_DEFAULT, zkConfig.getCuratorMaxRetries()); + assertEquals(expectedSessionTimeout, zkConfig.getSessionTimeoutMs()); + assertEquals(expectedConnectionTimeout, zkConfig.getConnectionTimeoutMs()); + } + + @Test + public void testCustomConfigurationValues() { + TezConfiguration conf = new TezConfiguration(); + conf.set(TezConfiguration.TEZ_AM_ZOOKEEPER_QUORUM, "zk1:2181,zk2:2181"); + conf.set(TezConfiguration.TEZ_AM_REGISTRY_NAMESPACE, "/custom-namespace"); + conf.set(TezConfiguration.TEZ_AM_CURATOR_BACKOFF_SLEEP, "2000ms"); + conf.setInt(TezConfiguration.TEZ_AM_CURATOR_MAX_RETRIES, 5); + conf.set(TezConfiguration.TEZ_AM_CURATOR_SESSION_TIMEOUT, "200000ms"); + conf.set(TezConfiguration.TEZ_AM_CURATOR_CONNECTION_TIMEOUT, "20000ms"); + + ZkConfig zkConfig = new ZkConfig(conf); + + assertEquals("zk1:2181,zk2:2181", zkConfig.getZkQuorum()); + assertEquals("/tez-external-sessions/custom-namespace", zkConfig.getZkNamespace()); + assertEquals(2000, zkConfig.getCuratorBackoffSleepMs()); + assertEquals(5, zkConfig.getCuratorMaxRetries()); + assertEquals(200000, zkConfig.getSessionTimeoutMs()); + assertEquals(20000, zkConfig.getConnectionTimeoutMs()); + } + + @Test + public void testNamespaceWithLeadingSlash() { + TezConfiguration conf = new TezConfiguration(); + conf.set(TezConfiguration.TEZ_AM_ZOOKEEPER_QUORUM, "localhost:2181"); + conf.set(TezConfiguration.TEZ_AM_REGISTRY_NAMESPACE, "/namespace-with-slash"); + + ZkConfig zkConfig = new ZkConfig(conf); + + assertEquals("/tez-external-sessions/namespace-with-slash", zkConfig.getZkNamespace()); + } + + @Test + public void testNamespaceWithoutLeadingSlash() { + TezConfiguration conf = new TezConfiguration(); + conf.set(TezConfiguration.TEZ_AM_ZOOKEEPER_QUORUM, "localhost:2181"); + conf.set(TezConfiguration.TEZ_AM_REGISTRY_NAMESPACE, "namespace-without-slash"); + + ZkConfig zkConfig = new ZkConfig(conf); + + assertEquals("/tez-external-sessions/namespace-without-slash", zkConfig.getZkNamespace()); + } + + @Test + public void testComputeGroupsDisabled() { + TezConfiguration conf = new TezConfiguration(); + conf.set(TezConfiguration.TEZ_AM_ZOOKEEPER_QUORUM, "localhost:2181"); + conf.set(TezConfiguration.TEZ_AM_REGISTRY_NAMESPACE, "/test-namespace"); + conf.setBoolean(TezConfiguration.TEZ_AM_REGISTRY_ENABLE_COMPUTE_GROUPS, false); + + ZkConfig zkConfig = new ZkConfig(conf); + + assertEquals("/tez-external-sessions/test-namespace", zkConfig.getZkNamespace()); + } + + @Test + public void testComputeGroupsEnabledWithoutEnvVar() { + TezConfiguration conf = new TezConfiguration(); + conf.set(TezConfiguration.TEZ_AM_ZOOKEEPER_QUORUM, "localhost:2181"); + conf.set(TezConfiguration.TEZ_AM_REGISTRY_NAMESPACE, "/test-namespace"); + conf.setBoolean(TezConfiguration.TEZ_AM_REGISTRY_ENABLE_COMPUTE_GROUPS, true); + + // When compute groups are enabled but env var is not set, namespace should not include sub-namespace + ZkConfig zkConfig = new ZkConfig(conf); + + // Namespace should start with base namespace (env var not set, so no sub-namespace added) + assertEquals("/tez-external-sessions/test-namespace", zkConfig.getZkNamespace()); + } + + @Test + public void testGetRetryPolicy() { + TezConfiguration conf = new TezConfiguration(); + conf.set(TezConfiguration.TEZ_AM_ZOOKEEPER_QUORUM, "localhost:2181"); + conf.set(TezConfiguration.TEZ_AM_CURATOR_BACKOFF_SLEEP, "1500ms"); + conf.setInt(TezConfiguration.TEZ_AM_CURATOR_MAX_RETRIES, 4); + + ZkConfig zkConfig = new ZkConfig(conf); + RetryPolicy retryPolicy = zkConfig.getRetryPolicy(); + + assertNotNull(retryPolicy); + // Verify it's an ExponentialBackoffRetry instance + assertEquals("org.apache.curator.retry.ExponentialBackoffRetry", retryPolicy.getClass().getName()); + } + + @Test + public void testTimeUnitSupport() { + TezConfiguration conf = new TezConfiguration(); + conf.set(TezConfiguration.TEZ_AM_ZOOKEEPER_QUORUM, "localhost:2181"); + // Test different time units + conf.set(TezConfiguration.TEZ_AM_CURATOR_BACKOFF_SLEEP, "2s"); + conf.set(TezConfiguration.TEZ_AM_CURATOR_SESSION_TIMEOUT, "3m"); + conf.set(TezConfiguration.TEZ_AM_CURATOR_CONNECTION_TIMEOUT, "5s"); + + ZkConfig zkConfig = new ZkConfig(conf); + + assertEquals(2000, zkConfig.getCuratorBackoffSleepMs()); + assertEquals(180000, zkConfig.getSessionTimeoutMs()); + assertEquals(5000, zkConfig.getConnectionTimeoutMs()); + + // Unit-less values should default to milliseconds + conf.set(TezConfiguration.TEZ_AM_CURATOR_BACKOFF_SLEEP, "2000"); + conf.set(TezConfiguration.TEZ_AM_CURATOR_SESSION_TIMEOUT, "300000"); + conf.set(TezConfiguration.TEZ_AM_CURATOR_CONNECTION_TIMEOUT, "15000"); + + ZkConfig unitlessConfig = new ZkConfig(conf); + assertEquals(2000, unitlessConfig.getCuratorBackoffSleepMs()); + assertEquals(300000, unitlessConfig.getSessionTimeoutMs()); + assertEquals(15000, unitlessConfig.getConnectionTimeoutMs()); + } + + @Test + public void testCreateCuratorFramework() { + TezConfiguration conf = new TezConfiguration(); + conf.set(TezConfiguration.TEZ_AM_ZOOKEEPER_QUORUM, "localhost:2181"); + + ZkConfig zkConfig = new ZkConfig(conf); + CuratorFramework curator = zkConfig.createCuratorFramework(); + + assertNotNull(curator); + assertEquals(zkConfig.getZkQuorum(), curator.getZookeeperClient().getCurrentConnectionString()); + } + + @Test(expected = IllegalArgumentException.class) + public void testNullZkQuorum() { + TezConfiguration conf = new TezConfiguration(); + // Don't set zkQuorum + new ZkConfig(conf); + } + + @Test(expected = IllegalArgumentException.class) + public void testEmptyZkQuorum() { + TezConfiguration conf = new TezConfiguration(); + conf.set(TezConfiguration.TEZ_AM_ZOOKEEPER_QUORUM, ""); + new ZkConfig(conf); + } + + @Test(expected = IllegalArgumentException.class) + public void testNullNamespace() { + TezConfiguration conf = new TezConfiguration(); + conf.set(TezConfiguration.TEZ_AM_ZOOKEEPER_QUORUM, "localhost:2181"); + conf.set(TezConfiguration.TEZ_AM_REGISTRY_NAMESPACE, null); + new ZkConfig(conf); + } + + @Test(expected = IllegalArgumentException.class) + public void testEmptyNamespace() { + TezConfiguration conf = new TezConfiguration(); + conf.set(TezConfiguration.TEZ_AM_ZOOKEEPER_QUORUM, "localhost:2181"); + conf.set(TezConfiguration.TEZ_AM_REGISTRY_NAMESPACE, ""); + new ZkConfig(conf); + } + + @Test + public void testDefaultNamespace() { + TezConfiguration conf = new TezConfiguration(); + conf.set(TezConfiguration.TEZ_AM_ZOOKEEPER_QUORUM, "localhost:2181"); + // Don't set namespace, should use default + ZkConfig zkConfig = new ZkConfig(conf); + assertEquals("/tez-external-sessions" + TezConfiguration.TEZ_AM_REGISTRY_NAMESPACE_DEFAULT, + zkConfig.getZkNamespace()); + } +} diff --git a/tez-api/src/test/java/org/apache/tez/client/registry/zookeeper/TestZkFrameworkClient.java b/tez-api/src/test/java/org/apache/tez/client/registry/zookeeper/TestZkFrameworkClient.java index ab4b3cb7d5..efb5f98733 100644 --- a/tez-api/src/test/java/org/apache/tez/client/registry/zookeeper/TestZkFrameworkClient.java +++ b/tez-api/src/test/java/org/apache/tez/client/registry/zookeeper/TestZkFrameworkClient.java @@ -28,10 +28,10 @@ import org.apache.curator.test.TestingServer; import org.apache.hadoop.registry.client.binding.RegistryUtils; import org.apache.hadoop.registry.client.types.ServiceRecord; +import org.apache.hadoop.test.LambdaTestUtils; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; -import org.apache.hadoop.yarn.api.records.YarnApplicationState; import org.apache.hadoop.yarn.client.api.YarnClientApplication; import org.apache.tez.client.registry.AMRecord; import org.apache.tez.dag.api.TezConfiguration; @@ -84,7 +84,7 @@ public void testInitAndLifecycle() throws Exception { zkFrameworkClient = new ZkFrameworkClient(); zkFrameworkClient.init(tezConf); - assertTrue("Client should be running after init", zkFrameworkClient.isRunning()); + assertFalse("Client should not be running after init", zkFrameworkClient.isRunning()); zkFrameworkClient.start(); assertTrue("Client should be running after start", zkFrameworkClient.isRunning()); @@ -103,16 +103,14 @@ public void testGetApplicationReportWithRegisteredAM() throws Exception { // Register a mock AM in ZooKeeper ApplicationId appId = ApplicationId.newInstance(System.currentTimeMillis(), 1); String testHostName = "test-host"; - String testHostIp = "127.0.0.1"; int testPort = 12345; - registerMockAM(tezConf, appId, testHostName, testHostIp, testPort); + registerMockAM(tezConf, appId, testHostName, testPort); zkFrameworkClient = new ZkFrameworkClient(); zkFrameworkClient.init(tezConf); zkFrameworkClient.start(); - // Give time for ZK registry to initialize - Thread.sleep(500); + LambdaTestUtils.await(1000, 100, () -> zkFrameworkClient.isZkInitialized()); ApplicationReport report = zkFrameworkClient.getApplicationReport(appId); @@ -120,8 +118,6 @@ public void testGetApplicationReportWithRegisteredAM() throws Exception { assertEquals("Application ID should match", appId, report.getApplicationId()); assertEquals("Host should match", testHostName, report.getHost()); assertEquals("Port should match", testPort, report.getRpcPort()); - assertEquals("Application state should be RUNNING", YarnApplicationState.RUNNING, - report.getYarnApplicationState()); assertEquals("AM host should be cached", testHostName, zkFrameworkClient.getAmHost()); assertEquals("AM port should be cached", testPort, zkFrameworkClient.getAmPort()); } @@ -137,16 +133,14 @@ public void testGetApplicationReportWithMissingAM() throws Exception { zkFrameworkClient.init(tezConf); zkFrameworkClient.start(); - // Give time for ZK registry to initialize - Thread.sleep(500); - ApplicationId appId = ApplicationId.newInstance(System.currentTimeMillis(), 1); + + LambdaTestUtils.await(1000, 100, () -> zkFrameworkClient.isZkInitialized()); + ApplicationReport report = zkFrameworkClient.getApplicationReport(appId); assertNotNull("Application report should not be null", report); assertEquals("Application ID should match", appId, report.getApplicationId()); - assertEquals("Application state should be FINISHED", YarnApplicationState.FINISHED, - report.getYarnApplicationState()); assertEquals("Final status should be FAILED", FinalApplicationStatus.FAILED, report.getFinalApplicationStatus()); assertTrue("Diagnostics should mention missing AM", @@ -162,14 +156,13 @@ public void testCreateApplication() throws Exception { // Register a mock AM in ZooKeeper ApplicationId appId = ApplicationId.newInstance(System.currentTimeMillis(), 1); - registerMockAM(tezConf, appId, "test-host", "127.0.0.1", 12345); + registerMockAM(tezConf, appId, "test-host", 12345); zkFrameworkClient = new ZkFrameworkClient(); zkFrameworkClient.init(tezConf); zkFrameworkClient.start(); - // Give time for ZK registry to initialize - Thread.sleep(500); + LambdaTestUtils.await(1000, 100, () -> zkFrameworkClient.isZkInitialized()); // Need to call getApplicationReport first to populate amRecord zkFrameworkClient.getApplicationReport(appId); @@ -214,10 +207,10 @@ private TezConfiguration createTezConf() { return tezConf; } - private void registerMockAM(TezConfiguration tezConf, ApplicationId appId, String hostName, String hostIp, int port) + private void registerMockAM(TezConfiguration tezConf, ApplicationId appId, String hostName, int port) throws Exception { // Create AM record and publish it directly to ZooKeeper - AMRecord amRecord = new AMRecord(appId, hostName, hostIp, port, "test-external-id", "test-compute"); + AMRecord amRecord = new AMRecord(appId, hostName, "127.0.0.1", port, "test-external-id", "test-compute"); ServiceRecord serviceRecord = amRecord.toServiceRecord(); RegistryUtils.ServiceRecordMarshal marshal = new RegistryUtils.ServiceRecordMarshal(); diff --git a/tez-dag/pom.xml b/tez-dag/pom.xml index 06e2340f06..ac5936206e 100644 --- a/tez-dag/pom.xml +++ b/tez-dag/pom.xml @@ -212,6 +212,7 @@ ${test.jvm.args} ${test.log.dir} + test-external-id diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/client/registry/zookeeper/ZkAMRegistry.java b/tez-dag/src/main/java/org/apache/tez/dag/api/client/registry/zookeeper/ZkAMRegistry.java index 83e9c8015f..b617070422 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/api/client/registry/zookeeper/ZkAMRegistry.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/api/client/registry/zookeeper/ZkAMRegistry.java @@ -28,10 +28,10 @@ import org.apache.curator.framework.CuratorFramework; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.registry.client.binding.RegistryUtils; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.tez.client.registry.AMRecord; import org.apache.tez.client.registry.AMRegistry; +import org.apache.tez.client.registry.AMRegistryUtils; import org.apache.tez.client.registry.zookeeper.ZkConfig; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; @@ -49,13 +49,12 @@ public class ZkAMRegistry extends AMRegistry { private static final Logger LOG = LoggerFactory.getLogger(ZkAMRegistry.class); - private final List amRecords = new ArrayList<>(); + private final List amRecords = Collections.synchronizedList(new ArrayList<>()); private final String externalId; private CuratorFramework client = null; private String namespace = null; private ZkConfig zkConfig = null; - private boolean started = false; public ZkAMRegistry(String externalId) { super("ZkAMRegistry"); @@ -64,47 +63,52 @@ public ZkAMRegistry(String externalId) { @Override public void serviceInit(Configuration conf) { - if (zkConfig == null) { - zkConfig = new ZkConfig(conf); - this.client = zkConfig.createCuratorFramework(); - this.namespace = zkConfig.getZkNamespace(); - LOG.info("AMRegistryZkImpl initialized"); - } + zkConfig = new ZkConfig(conf); + this.client = zkConfig.createCuratorFramework(); + this.namespace = zkConfig.getZkNamespace(); + LOG.info("ZkAMRegistry initialized"); } @Override public void serviceStart() throws Exception { - if (!started) { - client.start(); - started = true; - LOG.info("AMRegistryZkImpl started"); - } + client.start(); + LOG.info("ZkAMRegistry started"); } - //Deletes from Zookeeper AMRecords that were added by this instance + /** + * Shuts down the service by removing all {@link AMRecord} entries from ZooKeeper + * that were created by this instance. + * + *

After all removal attempts, the ZooKeeper client is closed and the shutdown + * is logged.

+ * + * @throws Exception if a failure occurs while closing the ZooKeeper client + */ @Override public void serviceStop() throws Exception { - List records = new ArrayList<>(amRecords); - for (AMRecord amRecord : records) { - remove(amRecord); + for (AMRecord amRecord : new ArrayList<>(amRecords)) { + try { + remove(amRecord); + } catch (Exception e) { + LOG.warn("Exception while trying to remove AMRecord: {}", amRecord, e); + } } client.close(); - LOG.info("AMRegistryZkImpl shutdown"); + LOG.info("ZkAMRegistry shutdown"); } //Serialize AMRecord to ServiceRecord and deliver the JSON bytes to //zkNode at the path: / @Override public void add(AMRecord server) throws Exception { - RegistryUtils.ServiceRecordMarshal marshal = new RegistryUtils.ServiceRecordMarshal(); - String json = marshal.toJson(server.toServiceRecord()); + String json = AMRegistryUtils.recordToJsonString(server); try { - final String path = namespace + "/" + server.getApplicationId().toString(); + final String path = pathFor(server); client.setData().forPath(path, json.getBytes(StandardCharsets.UTF_8)); LOG.info("Added AMRecord to zkpath {}", path); } catch (KeeperException.NoNodeException nne) { client.create().creatingParentContainersIfNeeded().withMode(CreateMode.EPHEMERAL) - .forPath(namespace + "/" + server.getApplicationId().toString(), json.getBytes(StandardCharsets.UTF_8)); + .forPath(pathFor(server), json.getBytes(StandardCharsets.UTF_8)); } amRecords.add(server); } @@ -112,7 +116,7 @@ public void add(AMRecord server) throws Exception { @Override public void remove(AMRecord server) throws Exception { amRecords.remove(server); - final String path = namespace + "/" + server.getApplicationId().toString(); + final String path = pathFor(server); client.delete().forPath(path); LOG.info("Deleted AMRecord from zkpath {}", path); } @@ -140,8 +144,10 @@ public ApplicationId generateNewId() throws Exception { .create() .withMode(CreateMode.EPHEMERAL) .forPath(namespace + "/" + tryAppId.toString(), new byte[0]); + LOG.debug("Successfully created application id {} for namespace {}", tryAppId, namespace); success = true; } catch (KeeperException.NodeExistsException nodeExists) { + LOG.info("Node already exists in ZK for application id {}", tryId); long elapsedTime = System.currentTimeMillis() - startTime; retryPolicy.allowRetry(i + 1, elapsedTime, RetryLoop.getDefaultRetrySleeper()); tryId++; @@ -172,4 +178,8 @@ private void createNamespaceIfNotExists() throws Exception { LOG.info("Namespace already exists, will use existing: {}", namespace); } } + + private String pathFor(AMRecord record) { + return namespace + "/" + record.getApplicationId().toString(); + } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java index 58d7ba928c..2ebffc6e1e 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java @@ -647,8 +647,7 @@ protected void serviceInit(final Configuration conf) throws Exception { } private static ServerFrameworkService getFrameworkService(Configuration conf) { - ServerFrameworkService frameworkService = FrameworkUtils.get(ServerFrameworkService.class, conf); - return frameworkService == null ? new YarnServerFrameworkService() : frameworkService; + return FrameworkUtils.get(ServerFrameworkService.class, conf, YarnServerFrameworkService.class); } @VisibleForTesting diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java index 941ea8fbd2..8ce4f6afa8 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java @@ -138,7 +138,6 @@ import org.apache.tez.dag.utils.RelocalizationUtils; import org.apache.tez.dag.utils.TaskSpecificLaunchCmdOption; import org.apache.tez.dag.utils.TezBuilderUtils; -import org.apache.tez.frameworkplugins.TaskResourceException; import org.apache.tez.runtime.api.OutputCommitter; import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; import org.apache.tez.state.OnStateChangedCallback; @@ -1616,7 +1615,7 @@ DAGState initializeDAG() { if (!appContext.isLocal()) { try { appContext.getAmExtensions().checkTaskResources(vertexMap, appContext.getClusterInfo()); - } catch (TaskResourceException e) { + } catch (Exception e) { LOG.error(e.getMessage()); addDiagnostic(e.getMessage()); finished(DAGState.FAILED); diff --git a/tez-dag/src/main/java/org/apache/tez/frameworkplugins/AMExtensions.java b/tez-dag/src/main/java/org/apache/tez/frameworkplugins/AMExtensions.java index 6c07960ba5..a758dfe01d 100644 --- a/tez-dag/src/main/java/org/apache/tez/frameworkplugins/AMExtensions.java +++ b/tez-dag/src/main/java/org/apache/tez/frameworkplugins/AMExtensions.java @@ -63,9 +63,9 @@ public interface AMExtensions { * * @param vertices mapping of vertex names to their DAG vertices * @param clusterInfo cluster resource information - * @throws TaskResourceException if resource requirements cannot be satisfied + * @throws Exception if resource requirements cannot be satisfied */ - void checkTaskResources(Map vertices, ClusterInfo clusterInfo) throws TaskResourceException; + void checkTaskResources(Map vertices, ClusterInfo clusterInfo) throws Exception; /** * Create or override the session token used for AM authentication. diff --git a/tez-dag/src/main/java/org/apache/tez/frameworkplugins/yarn/YarnServerFrameworkService.java b/tez-dag/src/main/java/org/apache/tez/frameworkplugins/yarn/YarnServerFrameworkService.java index 2ca4110932..40d6eb1d9c 100644 --- a/tez-dag/src/main/java/org/apache/tez/frameworkplugins/yarn/YarnServerFrameworkService.java +++ b/tez-dag/src/main/java/org/apache/tez/frameworkplugins/yarn/YarnServerFrameworkService.java @@ -29,7 +29,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.tez.client.registry.AMRegistry; import org.apache.tez.common.TezUtilsInternal; import org.apache.tez.common.security.JobTokenIdentifier; @@ -41,7 +40,6 @@ import org.apache.tez.dag.app.dag.Vertex; import org.apache.tez.frameworkplugins.AMExtensions; import org.apache.tez.frameworkplugins.ServerFrameworkService; -import org.apache.tez.frameworkplugins.TaskResourceException; /** * YARN-based server framework service implementation. @@ -78,11 +76,11 @@ public DAGProtos.ConfigurationProto loadConfigurationProto() throws IOException @Override public ContainerId allocateContainerId(Configuration conf) { String containerIdStr = System.getenv(ApplicationConstants.Environment.CONTAINER_ID.name()); - return ConverterUtils.toContainerId(containerIdStr); + return ContainerId.fromString(containerIdStr); } @Override - public void checkTaskResources(Map vertices, ClusterInfo clusterInfo) throws TaskResourceException { + public void checkTaskResources(Map vertices, ClusterInfo clusterInfo) throws Exception { Resource maxContainerCapability = clusterInfo.getMaxContainerCapability(); for (Vertex v : vertices.values()) { // TODO TEZ-2003 (post) TEZ-2624 Ideally, this should be per source. @@ -90,7 +88,7 @@ public void checkTaskResources(Map vertices, ClusterInfo cluster String msg = "Vertex's TaskResource is beyond the cluster container capability," + "Vertex=" + v.getLogIdentifier() +", Requested TaskResource=" + v.getTaskResource() + ", Cluster MaxContainerCapability=" + maxContainerCapability; - throw new TaskResourceException(msg); + throw new Exception(msg); } } } @@ -105,15 +103,9 @@ public Token getSessionToken( @Override public DAGProtos.PlanLocalResourcesProto getAdditionalSessionResources(String workingDirectory) throws IOException { - FileInputStream sessionResourcesStream = null; - try { - sessionResourcesStream = - new FileInputStream(new File(workingDirectory, TezConstants.TEZ_AM_LOCAL_RESOURCES_PB_FILE_NAME)); + try (FileInputStream sessionResourcesStream = new FileInputStream( + new File(workingDirectory, TezConstants.TEZ_AM_LOCAL_RESOURCES_PB_FILE_NAME))) { return DAGProtos.PlanLocalResourcesProto.parseDelimitedFrom(sessionResourcesStream); - } finally { - if (sessionResourcesStream != null) { - sessionResourcesStream.close(); - } } } } diff --git a/tez-dag/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZkStandaloneAMExtensions.java b/tez-dag/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZkStandaloneAMExtensions.java index ff3af7b495..6f95bcfb56 100644 --- a/tez-dag/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZkStandaloneAMExtensions.java +++ b/tez-dag/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZkStandaloneAMExtensions.java @@ -56,7 +56,7 @@ public ContainerId allocateContainerId(Configuration conf) { ApplicationAttemptId applicationAttemptId = ApplicationAttemptId.newInstance(appId, 1); return ContainerId.newContainerId(applicationAttemptId, 0); } else { - throw new RuntimeException("AMRegistry is required for ZkStandaloneAmExtensions"); + throw new IllegalStateException("AMRegistry must not be null for standalone AM mode"); } } catch (Exception e) { throw new RuntimeException(e); diff --git a/tez-dag/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZkStandaloneServerFrameworkService.java b/tez-dag/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZkStandaloneServerFrameworkService.java index dc8b7b91df..09ff1fb34b 100644 --- a/tez-dag/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZkStandaloneServerFrameworkService.java +++ b/tez-dag/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZkStandaloneServerFrameworkService.java @@ -30,17 +30,36 @@ public class ZkStandaloneServerFrameworkService implements ServerFrameworkService { private static final Logger LOG = LoggerFactory.getLogger(ZkStandaloneServerFrameworkService.class); - private ZkAMRegistry amRegistry; + private final ZkStandaloneAMExtensions amExtensions = new ZkStandaloneAMExtensions(this); + private volatile ZkAMRegistry amRegistry; + /** + * Returns a singleton {@link AMRegistry} instance backed by ZooKeeper. + * + *

If the registry has not yet been created, this method initializes and starts + * a new {@link ZkAMRegistry} using the external AM identifier obtained from the + * {@code TEZ_AM_EXTERNAL_ID} environment variable.

+ * + *

When the registry is used as a service within the DAGAppMaster, the + * DAGAppMaster is responsible for managing its lifecycle, including closure.

+ * + * @param conf the configuration used to initialize the registry; must not be null + * @return the initialized and started {@link AMRegistry} instance + * @throws IllegalStateException if the {@code TEZ_AM_EXTERNAL_ID} environment variable is not set + * @throws RuntimeException if an error occurs while creating, initializing, or starting the registry + */ @Override public synchronized AMRegistry getAMRegistry(Configuration conf) { if (amRegistry == null) { + final String externalId = System.getenv(TezConstants.TEZ_AM_EXTERNAL_ID); + if (externalId == null) { + throw new IllegalStateException( + TezConstants.TEZ_AM_EXTERNAL_ID + " environment variable is not set for standalone AM"); + } try { - final String externalID = System.getenv(TezConstants.TEZ_AM_EXTERNAL_ID); - amRegistry = new ZkAMRegistry(externalID); + amRegistry = new ZkAMRegistry(externalId); amRegistry.init(conf); - amRegistry.start(); - LOG.info("Created Zookeeper based AM Registry with externalID: {}", externalID); + LOG.info("Created Zookeeper based AM Registry with externalId: {}", externalId); } catch (Exception e) { throw new RuntimeException(e); } @@ -50,6 +69,6 @@ public synchronized AMRegistry getAMRegistry(Configuration conf) { @Override public AMExtensions getAMExtensions() { - return new ZkStandaloneAMExtensions(this); + return amExtensions; } } diff --git a/tez-dag/src/test/java/org/apache/tez/client/registry/zookeeper/TestZkAMRegistryClient.java b/tez-dag/src/test/java/org/apache/tez/client/registry/zookeeper/TestZkAMRegistryClient.java index 69fe2341c0..bbeee6b3c2 100644 --- a/tez-dag/src/test/java/org/apache/tez/client/registry/zookeeper/TestZkAMRegistryClient.java +++ b/tez-dag/src/test/java/org/apache/tez/client/registry/zookeeper/TestZkAMRegistryClient.java @@ -18,7 +18,12 @@ package org.apache.tez.client.registry.zookeeper; -import static org.junit.Assert.*; + +import static org.apache.tez.frameworkplugins.FrameworkMode.STANDALONE_ZOOKEEPER; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; import java.io.File; import java.util.List; @@ -105,7 +110,7 @@ public void teardown() throws Exception { *
  • Create and start a {@link ZkAMRegistryClient} with an event listener
  • *
  • Start a {@link MockDAGAppMaster} which registers itself to ZooKeeper
  • *
  • Verify that the registry client's listener is notified of the AM registration
  • - *
  • Verify the AM record can be retrieved via {@link ZkAMRegistryClient#getRecord(String)}
  • + *
  • Verify the AM record can be retrieved via {@link ZkAMRegistryClient#getRecord(ApplicationId)}
  • *
  • Verify the AM appears in the list from {@link ZkAMRegistryClient#getAllRecords()}
  • *
  • Validate all expected fields (host, port, applicationId) are correctly set
  • * @@ -164,10 +169,10 @@ public void onRemove(AMRecord amRecord) { assertTrue("AM was not discovered by registry client", amDiscovered.get()); // Verify the AM record is available through the registry client - AMRecord amRecord = registryClient.getRecord(appId.toString()); + AMRecord amRecord = registryClient.getRecord(appId); assertNotNull("AM record should be retrievable from registry", amRecord); assertEquals("Application ID should match", appId, amRecord.getApplicationId()); - assertNotNull("Host should be set", amRecord.getHost()); + assertNotNull("Host should be set", amRecord.getHostName()); assertTrue("Port should be positive", amRecord.getPort() > 0); // Verify getAllRecords also returns the AM @@ -183,12 +188,11 @@ public void onRemove(AMRecord amRecord) { } } assertTrue("AM record should be in getAllRecords", found); - LOG.info("Test completed successfully. AM was discovered: {}", amRecord); } private TezConfiguration getTezConfForZkDiscovery() { TezConfiguration tezConf = new TezConfiguration(); - tezConf.set(TezConfiguration.TEZ_FRAMEWORK_MODE, "STANDALONE_ZOOKEEPER"); + tezConf.set(TezConfiguration.TEZ_FRAMEWORK_MODE, STANDALONE_ZOOKEEPER.name()); tezConf.set(TezConfiguration.TEZ_AM_ZOOKEEPER_QUORUM, "localhost:" + zkServer.getPort()); tezConf.setBoolean(TezConfiguration.TEZ_LOCAL_MODE, true); tezConf.set(TezConfiguration.TEZ_AM_STAGING_DIR, TEST_DIR.toString()); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/api/client/registry/zookeeper/TestZkAMRegistry.java b/tez-dag/src/test/java/org/apache/tez/dag/api/client/registry/zookeeper/TestZkAMRegistry.java new file mode 100644 index 0000000000..cad6c4f1eb --- /dev/null +++ b/tez-dag/src/test/java/org/apache/tez/dag/api/client/registry/zookeeper/TestZkAMRegistry.java @@ -0,0 +1,192 @@ +/** + * 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.tez.dag.api.client.registry.zookeeper; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; + +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; + +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.test.TestingServer; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.tez.client.registry.AMRecord; +import org.apache.tez.client.registry.AMRegistryUtils; +import org.apache.tez.client.registry.zookeeper.ZkConfig; +import org.apache.tez.dag.api.TezConfiguration; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +/** + * Unit tests for {@link ZkAMRegistry}. + * + *

    This test class focuses on the low-level AM registry implementation that runs + * inside the AM process. It validates that:

    + *
      + *
    • Unique {@link ApplicationId}s are generated and persisted in ZooKeeper.
    • + *
    • {@link AMRecord}s are written to and removed from ZooKeeper at the expected paths.
    • + *
    + */ +public class TestZkAMRegistry { + + private TestingServer zkServer; + + @Before + public void setup() throws Exception { + zkServer = new TestingServer(); + zkServer.start(); + } + + @After + public void teardown() throws Exception { + if (zkServer != null) { + zkServer.close(); + } + } + + @Test + public void testGenerateNewIdProducesUniqueIds() throws Exception { + TezConfiguration conf = createTezConf(); + try (ZkAMRegistry registry = new ZkAMRegistry("external-id")) { + registry.serviceInit(conf); + registry.serviceStart(); + + ApplicationId first = registry.generateNewId(); + ApplicationId second = registry.generateNewId(); + + assertNotNull(first); + assertNotNull(second); + assertEquals("Cluster timestamps should match", first.getClusterTimestamp(), second.getClusterTimestamp()); + assertEquals("Second id should be first id + 1", first.getId() + 1, second.getId()); + } + } + + @Test(timeout = 120000) + public void testGenerateNewIdFromParallelThreads() throws Exception { + final int threadCount = 50; + + TezConfiguration conf = createTezConf(); + // this is the maxRetries for ExponentialBackoffRetry, let's use it to be able to test high concurrency + conf.setInt(TezConfiguration.TEZ_AM_CURATOR_MAX_RETRIES, 29); + + try (ZkAMRegistry registry = new ZkAMRegistry("external-id")) { + registry.serviceInit(conf); + registry.serviceStart(); + + ExecutorService executor = Executors.newFixedThreadPool(threadCount); + CountDownLatch startLatch = new CountDownLatch(1); + CountDownLatch doneLatch = new CountDownLatch(threadCount); + + Set ids = Collections.synchronizedSet(new HashSet<>()); + + List> asyncTasks = new ArrayList<>(); + + for (int i = 0; i < threadCount; i++) { + asyncTasks.add(CompletableFuture.runAsync(() -> { + try { + // Ensure all threads start generateNewId as simultaneously as possible + startLatch.await(); + ApplicationId id = registry.generateNewId(); + assertNotNull(id); + ids.add(id); + } catch (Exception e) { + throw new RuntimeException(e); + } finally { + doneLatch.countDown(); + } + }, executor)); + } + + // release all threads + startLatch.countDown(); + + // run the tasks + try { + CompletableFuture.allOf(asyncTasks.toArray(new CompletableFuture[0])).get(); + } catch (ExecutionException e) { // ExecutionException wraps the original exception + throw new RuntimeException(e.getCause()); + } finally { + executor.shutdown(); + } + assertEquals(String.format("All generated ids should be unique, ids found: %s", ids), threadCount, ids.size()); + + // additionally ensure cluster timestamp is the same for all IDs + long clusterTs = ids.iterator().next().getClusterTimestamp(); + for (ApplicationId id : ids) { + assertEquals("Cluster timestamps should match for all generated ids", clusterTs, id.getClusterTimestamp()); + } + } + } + + @Test + public void testAddAndRemoveAmRecordUpdatesZooKeeper() throws Exception { + TezConfiguration conf = createTezConf(); + + // Use a separate ZkConfig/Curator to inspect ZooKeeper state + ZkConfig zkConfig = new ZkConfig(conf); + + try (ZkAMRegistry registry = new ZkAMRegistry("external-id"); + CuratorFramework checkClient = zkConfig.createCuratorFramework()) { + registry.serviceInit(conf); + registry.serviceStart(); + + checkClient.start(); + + ApplicationId appId = registry.generateNewId(); + AMRecord record = registry.createAmRecord( + appId, "localhost", "127.0.0.1", 10000, "default-compute"); + + // Add record and verify node contents + registry.add(record); + + String path = zkConfig.getZkNamespace() + "/" + appId.toString(); + byte[] data = checkClient.getData().forPath(path); + + assertNotNull("Data should be written to ZooKeeper for AMRecord", data); + String json = new String(data, StandardCharsets.UTF_8); + String expectedJson = AMRegistryUtils.recordToJsonString(record); + assertEquals("Stored AMRecord JSON should match expected", expectedJson, json); + + // Remove record and ensure node is deleted + registry.remove(record); + assertNull("Node should be removed from ZooKeeper after remove()", checkClient.checkExists().forPath(path)); + } + } + + private TezConfiguration createTezConf() { + TezConfiguration conf = new TezConfiguration(); + conf.set(TezConfiguration.TEZ_AM_ZOOKEEPER_QUORUM, "localhost:" + zkServer.getPort()); + conf.set(TezConfiguration.TEZ_AM_REGISTRY_NAMESPACE, "/test-namespace"); + return conf; + } +} From 39387810bdb01e63d49522712e0b098cf39ad739 Mon Sep 17 00:00:00 2001 From: Laszlo Bodor Date: Mon, 1 Dec 2025 17:05:55 +0100 Subject: [PATCH 5/5] reworked am registry service lifecycle --- .../tez/client/registry/AMRegistry.java | 17 +++---- .../registry/zookeeper/ZkAMRegistry.java | 14 ++---- .../org/apache/tez/dag/app/DAGAppMaster.java | 50 ++++++++++++------- .../ZkStandaloneServerFrameworkService.java | 1 + .../registry/zookeeper/TestZkAMRegistry.java | 12 ++--- 5 files changed, 50 insertions(+), 44 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/client/registry/AMRegistry.java b/tez-api/src/main/java/org/apache/tez/client/registry/AMRegistry.java index bf87c5e9ec..ca0a19d5bf 100644 --- a/tez-api/src/main/java/org/apache/tez/client/registry/AMRegistry.java +++ b/tez-api/src/main/java/org/apache/tez/client/registry/AMRegistry.java @@ -19,7 +19,6 @@ package org.apache.tez.client.registry; -import org.apache.hadoop.service.AbstractService; import org.apache.hadoop.yarn.api.records.ApplicationId; @@ -38,18 +37,16 @@ * *

    */ -public abstract class AMRegistry extends AbstractService { - /* Implementations should provide a public no-arg constructor */ - protected AMRegistry(String name) { - super(name); - } +public interface AMRegistry extends AutoCloseable { - public abstract void add(AMRecord server) throws Exception; + void add(AMRecord server) throws Exception; - public abstract void remove(AMRecord server) throws Exception; + void remove(AMRecord server) throws Exception; - public abstract ApplicationId generateNewId() throws Exception; + ApplicationId generateNewId() throws Exception; - public abstract AMRecord createAmRecord(ApplicationId appId, String hostName, String hostIp, int port, + AMRecord createAmRecord(ApplicationId appId, String hostName, String hostIp, int port, String computeName); + + void close(); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/client/registry/zookeeper/ZkAMRegistry.java b/tez-dag/src/main/java/org/apache/tez/dag/api/client/registry/zookeeper/ZkAMRegistry.java index b617070422..005867ce16 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/api/client/registry/zookeeper/ZkAMRegistry.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/api/client/registry/zookeeper/ZkAMRegistry.java @@ -45,7 +45,7 @@ * Clients should use org.apache.tez.dag.api.client.registry.zookeeper.ZkAMRegistryClient instead. */ @InterfaceAudience.Private -public class ZkAMRegistry extends AMRegistry { +public class ZkAMRegistry implements AMRegistry { private static final Logger LOG = LoggerFactory.getLogger(ZkAMRegistry.class); @@ -57,20 +57,17 @@ public class ZkAMRegistry extends AMRegistry { private ZkConfig zkConfig = null; public ZkAMRegistry(String externalId) { - super("ZkAMRegistry"); this.externalId = externalId; } - @Override - public void serviceInit(Configuration conf) { + public void init(Configuration conf) { zkConfig = new ZkConfig(conf); this.client = zkConfig.createCuratorFramework(); this.namespace = zkConfig.getZkNamespace(); LOG.info("ZkAMRegistry initialized"); } - @Override - public void serviceStart() throws Exception { + public void start() throws Exception { client.start(); LOG.info("ZkAMRegistry started"); } @@ -81,11 +78,8 @@ public void serviceStart() throws Exception { * *

    After all removal attempts, the ZooKeeper client is closed and the shutdown * is logged.

    - * - * @throws Exception if a failure occurs while closing the ZooKeeper client */ - @Override - public void serviceStop() throws Exception { + public void close() { for (AMRecord amRecord : new ArrayList<>(amRecords)) { try { remove(amRecord); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java index 2ebffc6e1e..22497247dd 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java @@ -25,6 +25,7 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.io.PrintWriter; +import java.net.InetSocketAddress; import java.net.URI; import java.net.URISyntaxException; import java.net.URL; @@ -45,6 +46,7 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Objects; +import java.util.Optional; import java.util.Random; import java.util.Set; import java.util.Timer; @@ -618,11 +620,6 @@ protected void serviceInit(final Configuration conf) throws Exception { Executors.newFixedThreadPool(threadCount, new ThreadFactoryBuilder() .setDaemon(true).setNameFormat("App Shared Pool - #%d").build()); execService = MoreExecutors.listeningDecorator(rawExecutor); - AMRegistry amRegistry = frameworkService.getAMRegistry(conf); - if (amRegistry != null) { - initAmRegistry(appAttemptID.getApplicationId(), amRegistry, clientRpcServer); - addIfService(amRegistry, false); - } initServices(conf); super.serviceInit(conf); @@ -650,16 +647,35 @@ private static ServerFrameworkService getFrameworkService(Configuration conf) { return FrameworkUtils.get(ServerFrameworkService.class, conf, YarnServerFrameworkService.class); } - @VisibleForTesting - public void initAmRegistry(ApplicationId appId, AMRegistry amRegistry, DAGClientServer dagClientServer) { - dagClientServer.registerServiceListener((service) -> { + protected void initClientRpcServer() { + clientRpcServer = new DAGClientServer(clientHandler, appAttemptID, recoveryFS); + addIfService(clientRpcServer, true); + + initAmRegistryCallbackForRecordAdd(); + } + + /** + * Initializes an AM registry callback when the clientRpcServer is already initialized + */ + private void initAmRegistryCallbackForRecordAdd() { + AMRegistry amRegistry = frameworkService.getAMRegistry(this.amConf); + if (amRegistry == null) { + return; + } + if (clientRpcServer == null){ + throw new IllegalStateException( + "Client RPC Server has not been initialized before attempting to initialize an AM registry"); + } + + ApplicationId appId = appAttemptID.getApplicationId(); + + clientRpcServer.registerServiceListener((service) -> { if (service.isInState(STATE.STARTED)) { + InetSocketAddress rpcServerAddress = clientRpcServer.getBindAddress(); + final String computeName = System.getenv(ZkConfig.COMPUTE_GROUP_NAME_ENV); - AMRecord amRecord = amRegistry.createAmRecord( - appId, dagClientServer.getBindAddress().getHostName(), - dagClientServer.getBindAddress().getAddress().getHostAddress(), - dagClientServer.getBindAddress().getPort(), computeName - ); + AMRecord amRecord = amRegistry.createAmRecord(appId, rpcServerAddress.getHostName(), + rpcServerAddress.getAddress().getHostAddress(), rpcServerAddress.getPort(), computeName); try { amRegistry.add(amRecord); LOG.info("Added AMRecord: {} to registry..", amRecord); @@ -670,11 +686,6 @@ public void initAmRegistry(ApplicationId appId, AMRegistry amRegistry, DAGClient }); } - protected void initClientRpcServer() { - clientRpcServer = new DAGClientServer(clientHandler, appAttemptID, recoveryFS); - addIfService(clientRpcServer, true); - } - @VisibleForTesting protected DAGAppMasterShutdownHandler createShutdownHandler() { return new DAGAppMasterShutdownHandler(); @@ -1967,6 +1978,9 @@ void stopServices() { firstException = ex; } } + + Optional.ofNullable(frameworkService.getAMRegistry(this.amConf)).ifPresent(AMRegistry::close); + //after stopping all services, rethrow the first exception raised if (firstException != null) { throw ServiceStateException.convert(firstException); diff --git a/tez-dag/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZkStandaloneServerFrameworkService.java b/tez-dag/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZkStandaloneServerFrameworkService.java index 09ff1fb34b..12c62640ac 100644 --- a/tez-dag/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZkStandaloneServerFrameworkService.java +++ b/tez-dag/src/main/java/org/apache/tez/frameworkplugins/zookeeper/ZkStandaloneServerFrameworkService.java @@ -59,6 +59,7 @@ public synchronized AMRegistry getAMRegistry(Configuration conf) { try { amRegistry = new ZkAMRegistry(externalId); amRegistry.init(conf); + amRegistry.start(); LOG.info("Created Zookeeper based AM Registry with externalId: {}", externalId); } catch (Exception e) { throw new RuntimeException(e); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/api/client/registry/zookeeper/TestZkAMRegistry.java b/tez-dag/src/test/java/org/apache/tez/dag/api/client/registry/zookeeper/TestZkAMRegistry.java index cad6c4f1eb..cf3b9872e0 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/api/client/registry/zookeeper/TestZkAMRegistry.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/api/client/registry/zookeeper/TestZkAMRegistry.java @@ -78,8 +78,8 @@ public void teardown() throws Exception { public void testGenerateNewIdProducesUniqueIds() throws Exception { TezConfiguration conf = createTezConf(); try (ZkAMRegistry registry = new ZkAMRegistry("external-id")) { - registry.serviceInit(conf); - registry.serviceStart(); + registry.init(conf); + registry.start(); ApplicationId first = registry.generateNewId(); ApplicationId second = registry.generateNewId(); @@ -100,8 +100,8 @@ public void testGenerateNewIdFromParallelThreads() throws Exception { conf.setInt(TezConfiguration.TEZ_AM_CURATOR_MAX_RETRIES, 29); try (ZkAMRegistry registry = new ZkAMRegistry("external-id")) { - registry.serviceInit(conf); - registry.serviceStart(); + registry.init(conf); + registry.start(); ExecutorService executor = Executors.newFixedThreadPool(threadCount); CountDownLatch startLatch = new CountDownLatch(1); @@ -157,8 +157,8 @@ public void testAddAndRemoveAmRecordUpdatesZooKeeper() throws Exception { try (ZkAMRegistry registry = new ZkAMRegistry("external-id"); CuratorFramework checkClient = zkConfig.createCuratorFramework()) { - registry.serviceInit(conf); - registry.serviceStart(); + registry.init(conf); + registry.start(); checkClient.start();