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:
+ *
+ *
+ * - Configure Tez with STANDALONE_ZOOKEEPER framework mode
+ * - 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 appears in the list from {@link ZkAMRegistryClient#getAllRecords()}
+ * - Validate all expected fields (host, port, applicationId) are correctly set
+ *
+ *
+ * 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();