Merge r1569890 through r1571508 from trunk.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-5535@1571509 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Tsz-wo Sze 2014-02-25 01:16:30 +00:00
commit 0809f32f4b
58 changed files with 1871 additions and 1039 deletions

View File

@ -124,6 +124,8 @@ Trunk (Unreleased)
HADOOP-10342. Add a new method to UGI to use a Kerberos login subject to
build a new UGI. (Larry McCay via omalley)
HADOOP-9968. Makes ProxyUsers to work with NetGroups (Benoy Antony via ddas)
BUG FIXES
HADOOP-9451. Fault single-layer config if node group topology is enabled.

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.security.authorize;
import java.net.InetAddress;
import java.net.UnknownHostException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.Map;
@ -27,6 +28,7 @@ import java.util.Map.Entry;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.Groups;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.StringUtils;
@ -68,8 +70,11 @@ public class ProxyUsers {
String regex = CONF_HADOOP_PROXYUSER_RE+"[^.]*\\"+CONF_GROUPS;
Map<String,String> allMatchKeys = conf.getValByRegex(regex);
for(Entry<String, String> entry : allMatchKeys.entrySet()) {
proxyGroups.put(entry.getKey(),
StringUtils.getStringCollection(entry.getValue()));
Collection<String> groups = StringUtils.getStringCollection(entry.getValue());
proxyGroups.put(entry.getKey(), groups );
//cache the groups. This is needed for NetGroups
Groups.getUserToGroupsMappingService(conf).cacheGroupsAdd(
new ArrayList<String>(groups));
}
// now hosts

View File

@ -17,23 +17,113 @@
*/
package org.apache.hadoop.security.authorize;
import java.io.IOException;
import java.util.Arrays;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.util.NativeCodeLoader;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.security.Groups;
import org.apache.hadoop.security.UserGroupInformation;
import org.junit.Test;
import static org.junit.Assert.*;
public class TestProxyUsers {
private static final Log LOG =
LogFactory.getLog(TestProxyUsers.class);
private static final String REAL_USER_NAME = "proxier";
private static final String PROXY_USER_NAME = "proxied_user";
private static final String[] GROUP_NAMES =
new String[] { "foo_group" };
private static final String[] NETGROUP_NAMES =
new String[] { "@foo_group" };
private static final String[] OTHER_GROUP_NAMES =
new String[] { "bar_group" };
private static final String PROXY_IP = "1.2.3.4";
/**
* Test the netgroups (groups in ACL rules that start with @)
*
* This is a manual test because it requires:
* - host setup
* - native code compiled
* - specify the group mapping class
*
* Host setup:
*
* /etc/nsswitch.conf should have a line like this:
* netgroup: files
*
* /etc/netgroup should be (the whole file):
* foo_group (,proxied_user,)
*
* To run this test:
*
* export JAVA_HOME='path/to/java'
* mvn test \
* -Dtest=TestProxyUsers \
* -DTestProxyUsersGroupMapping=$className \
*
* where $className is one of the classes that provide group
* mapping services, i.e. classes that implement
* GroupMappingServiceProvider interface, at this time:
* - org.apache.hadoop.security.JniBasedUnixGroupsNetgroupMapping
* - org.apache.hadoop.security.ShellBasedUnixGroupsNetgroupMapping
*
*/
@Test
public void testNetgroups () throws IOException{
if(!NativeCodeLoader.isNativeCodeLoaded()) {
LOG.info("Not testing netgroups, " +
"this test only runs when native code is compiled");
return;
}
String groupMappingClassName =
System.getProperty("TestProxyUsersGroupMapping");
if(groupMappingClassName == null) {
LOG.info("Not testing netgroups, no group mapping class specified, " +
"use -DTestProxyUsersGroupMapping=$className to specify " +
"group mapping class (must implement GroupMappingServiceProvider " +
"interface and support netgroups)");
return;
}
LOG.info("Testing netgroups using: " + groupMappingClassName);
Configuration conf = new Configuration();
conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_GROUP_MAPPING,
groupMappingClassName);
conf.set(
ProxyUsers.getProxySuperuserGroupConfKey(REAL_USER_NAME),
StringUtils.join(",", Arrays.asList(NETGROUP_NAMES)));
conf.set(
ProxyUsers.getProxySuperuserIpConfKey(REAL_USER_NAME),
PROXY_IP);
ProxyUsers.refreshSuperUserGroupsConfiguration(conf);
Groups groups = Groups.getUserToGroupsMappingService(conf);
// try proxying a group that's allowed
UserGroupInformation realUserUgi = UserGroupInformation
.createRemoteUser(REAL_USER_NAME);
UserGroupInformation proxyUserUgi = UserGroupInformation.createProxyUserForTesting(
PROXY_USER_NAME, realUserUgi, groups.getGroups(PROXY_USER_NAME).toArray(
new String[groups.getGroups(PROXY_USER_NAME).size()]));
assertAuthorized(proxyUserUgi, PROXY_IP);
}
@Test
public void testProxyUsers() throws Exception {
Configuration conf = new Configuration();

View File

@ -353,6 +353,8 @@ Release 2.4.0 - UNRELEASED
HDFS-5698. Use protobuf to serialize / deserialize FSImage. (See breakdown
of tasks below for features and contributors)
HDFS-5776 Support 'hedged' reads in DFSClient (Liang Xie via stack)
IMPROVEMENTS
HDFS-5781. Use an array to record the mapping between FSEditLogOpCode and

View File

@ -80,6 +80,10 @@ import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.concurrent.SynchronousQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import javax.net.SocketFactory;
@ -175,6 +179,7 @@ import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenRenewer;
import org.apache.hadoop.util.Daemon;
import org.apache.hadoop.util.DataChecksum;
import org.apache.hadoop.util.DataChecksum.Type;
import org.apache.hadoop.util.Progressable;
@ -224,6 +229,10 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory {
private final CachingStrategy defaultReadCachingStrategy;
private final CachingStrategy defaultWriteCachingStrategy;
private final ClientContext clientContext;
private volatile long hedgedReadThresholdMillis;
private static DFSHedgedReadMetrics HEDGED_READ_METRIC =
new DFSHedgedReadMetrics();
private static ThreadPoolExecutor HEDGED_READ_THREAD_POOL;
/**
* DFSClient configuration
@ -576,6 +585,15 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory {
this.clientContext = ClientContext.get(
conf.get(DFS_CLIENT_CONTEXT, DFS_CLIENT_CONTEXT_DEFAULT),
dfsClientConf);
this.hedgedReadThresholdMillis = conf.getLong(
DFSConfigKeys.DFS_DFSCLIENT_HEDGED_READ_THRESHOLD_MILLIS,
DFSConfigKeys.DEFAULT_DFSCLIENT_HEDGED_READ_THRESHOLD_MILLIS);
int numThreads = conf.getInt(
DFSConfigKeys.DFS_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE,
DFSConfigKeys.DEFAULT_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE);
if (numThreads > 0) {
this.initThreadsNumForHedgedReads(numThreads);
}
}
/**
@ -2720,4 +2738,64 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory {
}
}
}
/**
* Create hedged reads thread pool, HEDGED_READ_THREAD_POOL, if
* it does not already exist.
* @param num Number of threads for hedged reads thread pool.
* If zero, skip hedged reads thread pool creation.
*/
private synchronized void initThreadsNumForHedgedReads(int num) {
if (num <= 0 || HEDGED_READ_THREAD_POOL != null) return;
HEDGED_READ_THREAD_POOL = new ThreadPoolExecutor(1, num, 60,
TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
new Daemon.DaemonFactory() {
private final AtomicInteger threadIndex =
new AtomicInteger(0);
@Override
public Thread newThread(Runnable r) {
Thread t = super.newThread(r);
t.setName("hedgedRead-" +
threadIndex.getAndIncrement());
return t;
}
},
new ThreadPoolExecutor.CallerRunsPolicy() {
@Override
public void rejectedExecution(Runnable runnable,
ThreadPoolExecutor e) {
LOG.info("Execution rejected, Executing in current thread");
HEDGED_READ_METRIC.incHedgedReadOpsInCurThread();
// will run in the current thread
super.rejectedExecution(runnable, e);
}
});
HEDGED_READ_THREAD_POOL.allowCoreThreadTimeOut(true);
if (LOG.isDebugEnabled()) {
LOG.debug("Using hedged reads; pool threads=" + num);
}
}
long getHedgedReadTimeout() {
return this.hedgedReadThresholdMillis;
}
@VisibleForTesting
void setHedgedReadTimeout(long timeoutMillis) {
this.hedgedReadThresholdMillis = timeoutMillis;
}
ThreadPoolExecutor getHedgedReadsThreadPool() {
return HEDGED_READ_THREAD_POOL;
}
boolean isHedgedReadsEnabled() {
return (HEDGED_READ_THREAD_POOL != null) &&
HEDGED_READ_THREAD_POOL.getMaximumPoolSize() > 0;
}
DFSHedgedReadMetrics getHedgedReadMetrics() {
return HEDGED_READ_METRIC;
}
}

View File

@ -16,7 +16,6 @@
* limitations under the License.
*/
package org.apache.hadoop.hdfs;
import java.io.IOException;
import com.google.common.annotations.VisibleForTesting;
@ -46,4 +45,6 @@ public class DFSClientFaultInjector {
public boolean failPacket() {
return false;
}
public void startFetchFromDatanode() {}
}

View File

@ -595,4 +595,14 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
public static final int DFS_HTTP_CLIENT_FAILOVER_SLEEPTIME_BASE_DEFAULT = 500;
public static final String DFS_HTTP_CLIENT_FAILOVER_SLEEPTIME_MAX_KEY = "dfs.http.client.failover.sleep.max.millis";
public static final int DFS_HTTP_CLIENT_FAILOVER_SLEEPTIME_MAX_DEFAULT = 15000;
// hedged read properties
public static final String DFS_DFSCLIENT_HEDGED_READ_THRESHOLD_MILLIS =
"dfs.client.hedged.read.threshold.millis";
public static final long DEFAULT_DFSCLIENT_HEDGED_READ_THRESHOLD_MILLIS =
500;
public static final String DFS_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE =
"dfs.client.hedged.read.threadpool.size";
public static final int DEFAULT_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE = 0;
}

View File

@ -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.hadoop.hdfs;
import java.util.concurrent.atomic.AtomicLong;
/**
* The client-side metrics for hedged read feature.
* This class has a number of metrics variables that are publicly accessible,
* we can grab them from client side, like HBase.
*/
public class DFSHedgedReadMetrics {
public AtomicLong hedgedReadOps = new AtomicLong();
public AtomicLong hedgedReadOpsWin = new AtomicLong();
public AtomicLong hedgedReadOpsInCurThread = new AtomicLong();
public void incHedgedReadOps() {
hedgedReadOps.incrementAndGet();
}
public void incHedgedReadOpsInCurThread() {
hedgedReadOpsInCurThread.incrementAndGet();
}
public void incHedgedReadWins() {
hedgedReadOpsWin.incrementAndGet();
}
public long getHedgedReadOps() {
return hedgedReadOps.longValue();
}
public long getHedgedReadOpsInCurThread() {
return hedgedReadOpsInCurThread.longValue();
}
public long getHedgedReadWins() {
return hedgedReadOpsWin.longValue();
}
}

View File

@ -17,13 +17,12 @@
*/
package org.apache.hadoop.hdfs;
import java.io.FileInputStream;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.net.Socket;
import java.nio.ByteBuffer;
import java.util.AbstractMap;
import java.util.ArrayList;
import java.util.Collection;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.HashSet;
@ -32,9 +31,14 @@ import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.ByteBufferReadable;
import org.apache.hadoop.fs.ByteBufferUtil;
@ -54,15 +58,12 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
import org.apache.hadoop.io.ByteBufferPool;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.net.unix.DomainSocket;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.IdentityHashStore;
@ -555,7 +556,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
assert (target==pos) : "Wrong postion " + pos + " expect " + target;
long offsetIntoBlock = target - targetBlock.getStartOffset();
DNAddrPair retval = chooseDataNode(targetBlock);
DNAddrPair retval = chooseDataNode(targetBlock, null);
chosenNode = retval.info;
InetSocketAddress targetAddr = retval.addr;
@ -863,32 +864,30 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
corruptedBlockMap.put(blk, dnSet);
}
}
private DNAddrPair chooseDataNode(LocatedBlock block)
throws IOException {
private DNAddrPair chooseDataNode(LocatedBlock block,
Collection<DatanodeInfo> ignoredNodes) throws IOException {
while (true) {
DatanodeInfo[] nodes = block.getLocations();
try {
DatanodeInfo chosenNode = bestNode(nodes, deadNodes);
final String dnAddr =
chosenNode.getXferAddr(dfsClient.getConf().connectToDnViaHostname);
if (DFSClient.LOG.isDebugEnabled()) {
DFSClient.LOG.debug("Connecting to datanode " + dnAddr);
}
InetSocketAddress targetAddr = NetUtils.createSocketAddr(dnAddr);
return new DNAddrPair(chosenNode, targetAddr);
return getBestNodeDNAddrPair(nodes, ignoredNodes);
} catch (IOException ie) {
String errMsg =
getBestNodeDNAddrPairErrorString(nodes, deadNodes, ignoredNodes);
String blockInfo = block.getBlock() + " file=" + src;
if (failures >= dfsClient.getMaxBlockAcquireFailures()) {
throw new BlockMissingException(src, "Could not obtain block: " + blockInfo,
block.getStartOffset());
String description = "Could not obtain block: " + blockInfo;
DFSClient.LOG.warn(description + errMsg
+ ". Throwing a BlockMissingException");
throw new BlockMissingException(src, description,
block.getStartOffset());
}
if (nodes == null || nodes.length == 0) {
DFSClient.LOG.info("No node available for " + blockInfo);
}
DFSClient.LOG.info("Could not obtain " + block.getBlock()
+ " from any node: " + ie
+ " from any node: " + ie + errMsg
+ ". Will get new block locations from namenode and retry...");
try {
// Introducing a random factor to the wait time before another retry.
@ -914,21 +913,99 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
continue;
}
}
}
}
/**
* Get the best node.
* @param nodes Nodes to choose from.
* @param ignoredNodes Do not chose nodes in this array (may be null)
* @return The DNAddrPair of the best node.
* @throws IOException
*/
private DNAddrPair getBestNodeDNAddrPair(final DatanodeInfo[] nodes,
Collection<DatanodeInfo> ignoredNodes) throws IOException {
DatanodeInfo chosenNode = bestNode(nodes, deadNodes, ignoredNodes);
final String dnAddr =
chosenNode.getXferAddr(dfsClient.getConf().connectToDnViaHostname);
if (DFSClient.LOG.isDebugEnabled()) {
DFSClient.LOG.debug("Connecting to datanode " + dnAddr);
}
InetSocketAddress targetAddr = NetUtils.createSocketAddr(dnAddr);
return new DNAddrPair(chosenNode, targetAddr);
}
private static String getBestNodeDNAddrPairErrorString(
DatanodeInfo nodes[], AbstractMap<DatanodeInfo,
DatanodeInfo> deadNodes, Collection<DatanodeInfo> ignoredNodes) {
StringBuilder errMsgr = new StringBuilder(
" No live nodes contain current block ");
errMsgr.append("Block locations:");
for (DatanodeInfo datanode : nodes) {
errMsgr.append(" ");
errMsgr.append(datanode.toString());
}
errMsgr.append(" Dead nodes: ");
for (DatanodeInfo datanode : deadNodes.keySet()) {
errMsgr.append(" ");
errMsgr.append(datanode.toString());
}
if (ignoredNodes != null) {
errMsgr.append(" Ignored nodes: ");
for (DatanodeInfo datanode : ignoredNodes) {
errMsgr.append(" ");
errMsgr.append(datanode.toString());
}
}
return errMsgr.toString();
}
private void fetchBlockByteRange(LocatedBlock block, long start, long end,
byte[] buf, int offset,
Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
throws IOException {
//
// Connect to best DataNode for desired Block, with potential offset
//
block = getBlockAt(block.getStartOffset(), false);
while (true) {
DNAddrPair addressPair = chooseDataNode(block, null);
try {
actualGetFromOneDataNode(addressPair, block, start, end, buf, offset,
corruptedBlockMap);
return;
} catch (IOException e) {
// Ignore. Already processed inside the function.
// Loop through to try the next node.
}
}
}
private Callable<ByteBuffer> getFromOneDataNode(final DNAddrPair datanode,
final LocatedBlock block, final long start, final long end,
final ByteBuffer bb,
final Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap,
final CountDownLatch latch) {
return new Callable<ByteBuffer>() {
@Override
public ByteBuffer call() throws Exception {
byte[] buf = bb.array();
int offset = bb.position();
actualGetFromOneDataNode(datanode, block, start, end, buf, offset,
corruptedBlockMap);
latch.countDown();
return bb;
}
};
}
private void actualGetFromOneDataNode(final DNAddrPair datanode,
LocatedBlock block, final long start, final long end, byte[] buf,
int offset, Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
throws IOException {
DFSClientFaultInjector.get().startFetchFromDatanode();
int refetchToken = 1; // only need to get a new access token once
int refetchEncryptionKey = 1; // only need to get a new encryption key once
while (true) {
// cached block locations may have been updated by chooseDataNode()
// or fetchBlockAt(). Always get the latest list of locations at the
// or fetchBlockAt(). Always get the latest list of locations at the
// start of the loop.
CachingStrategy curCachingStrategy;
boolean allowShortCircuitLocalReads;
@ -937,11 +1014,10 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
curCachingStrategy = cachingStrategy;
allowShortCircuitLocalReads = !shortCircuitForbidden();
}
DNAddrPair retval = chooseDataNode(block);
DatanodeInfo chosenNode = retval.info;
InetSocketAddress targetAddr = retval.addr;
DatanodeInfo chosenNode = datanode.info;
InetSocketAddress targetAddr = datanode.addr;
BlockReader reader = null;
try {
Token<BlockTokenIdentifier> blockToken = block.getBlockToken();
int len = (int) (end - start + 1);
@ -969,11 +1045,14 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
}
return;
} catch (ChecksumException e) {
DFSClient.LOG.warn("fetchBlockByteRange(). Got a checksum exception for " +
src + " at " + block.getBlock() + ":" +
e.getPos() + " from " + chosenNode);
String msg = "fetchBlockByteRange(). Got a checksum exception for "
+ src + " at " + block.getBlock() + ":" + e.getPos() + " from "
+ chosenNode;
DFSClient.LOG.warn(msg);
// we want to remember what we have tried
addIntoCorruptedBlockMap(block.getBlock(), chosenNode, corruptedBlockMap);
addToDeadNodes(chosenNode);
throw new IOException(msg);
} catch (IOException e) {
if (e instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
DFSClient.LOG.info("Will fetch a new encryption key and retry, "
@ -985,22 +1064,164 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
continue;
} else if (refetchToken > 0 && tokenRefetchNeeded(e, targetAddr)) {
refetchToken--;
fetchBlockAt(block.getStartOffset());
try {
fetchBlockAt(block.getStartOffset());
} catch (IOException fbae) {
// ignore IOE, since we can retry it later in a loop
}
continue;
} else {
DFSClient.LOG.warn("Failed to connect to " + targetAddr +
" for file " + src + " for block " + block.getBlock() + ":" + e);
if (DFSClient.LOG.isDebugEnabled()) {
DFSClient.LOG.debug("Connection failure ", e);
}
String msg = "Failed to connect to " + targetAddr + " for file "
+ src + " for block " + block.getBlock() + ":" + e;
DFSClient.LOG.warn("Connection failure: " + msg, e);
addToDeadNodes(chosenNode);
throw new IOException(msg);
}
} finally {
if (reader != null) {
reader.close();
}
}
// Put chosen node into dead list, continue
addToDeadNodes(chosenNode);
}
}
/**
* Like {@link #fetchBlockByteRange(LocatedBlock, long, long, byte[],
* int, Map)} except we start up a second, parallel, 'hedged' read
* if the first read is taking longer than configured amount of
* time. We then wait on which ever read returns first.
*
* @param block
* @param start
* @param end
* @param buf
* @param offset
* @param corruptedBlockMap
* @throws IOException
*/
private void hedgedFetchBlockByteRange(LocatedBlock block, long start,
long end, byte[] buf, int offset,
Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
throws IOException {
ArrayList<Future<ByteBuffer>> futures = null;
ArrayList<DatanodeInfo> ignored = new ArrayList<DatanodeInfo>();
ByteBuffer bb = null;
int len = (int) (end - start + 1);
block = getBlockAt(block.getStartOffset(), false);
// Latch shared by all outstanding reads. First to finish closes
CountDownLatch hasReceivedResult = new CountDownLatch(1);
while (true) {
DNAddrPair chosenNode = null;
Future<ByteBuffer> future = null;
// futures is null if there is no request already executing.
if (futures == null) {
// chooseDataNode is a commitment. If no node, we go to
// the NN to reget block locations. Only go here on first read.
chosenNode = chooseDataNode(block, ignored);
bb = ByteBuffer.wrap(buf, offset, len);
future = getHedgedReadFuture(chosenNode, block, start, end, bb,
corruptedBlockMap, hasReceivedResult);
try {
future.get(dfsClient.getHedgedReadTimeout(), TimeUnit.MILLISECONDS);
return;
} catch (TimeoutException e) {
if (DFSClient.LOG.isDebugEnabled()) {
DFSClient.LOG.debug("Waited " + dfsClient.getHedgedReadTimeout() +
"ms to read from " + chosenNode.info + "; spawning hedged read");
}
// Ignore this node on next go around.
ignored.add(chosenNode.info);
dfsClient.getHedgedReadMetrics().incHedgedReadOps();
futures = new ArrayList<Future<ByteBuffer>>();
futures.add(future);
continue; // no need to refresh block locations
} catch (InterruptedException e) {
// Ignore
} catch (ExecutionException e) {
// Ignore already logged in the call.
}
} else {
// We are starting up a 'hedged' read. We have a read already
// ongoing. Call getBestNodeDNAddrPair instead of chooseDataNode.
// If no nodes to do hedged reads against, pass.
try {
chosenNode = getBestNodeDNAddrPair(block.getLocations(), ignored);
bb = ByteBuffer.allocate(len);
future = getHedgedReadFuture(chosenNode, block, start, end, bb,
corruptedBlockMap, hasReceivedResult);
futures.add(future);
} catch (IOException ioe) {
if (DFSClient.LOG.isDebugEnabled()) {
DFSClient.LOG.debug("Failed getting node for hedged read: " +
ioe.getMessage());
}
}
// if not succeeded. Submit callables for each datanode in a loop, wait
// for a fixed interval and get the result from the fastest one.
try {
ByteBuffer result = getFirstToComplete(futures, hasReceivedResult);
// cancel the rest.
cancelAll(futures);
if (result.array() != buf) { // compare the array pointers
dfsClient.getHedgedReadMetrics().incHedgedReadWins();
System.arraycopy(result.array(), result.position(), buf, offset,
len);
} else {
dfsClient.getHedgedReadMetrics().incHedgedReadOps();
}
return;
} catch (InterruptedException ie) {
// Ignore
} catch (ExecutionException e) {
// exception already handled in the call method. getFirstToComplete
// will remove the failing future from the list. nothing more to do.
}
// We got here if exception. Ignore this node on next go around.
ignored.add(chosenNode.info);
}
// executed if we get an error from a data node
block = getBlockAt(block.getStartOffset(), false);
}
}
private Future<ByteBuffer> getHedgedReadFuture(final DNAddrPair chosenNode,
final LocatedBlock block, long start,
final long end, final ByteBuffer bb,
final Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap,
final CountDownLatch hasReceivedResult) {
Callable<ByteBuffer> getFromDataNodeCallable =
getFromOneDataNode(chosenNode, block, start, end, bb,
corruptedBlockMap, hasReceivedResult);
return dfsClient.getHedgedReadsThreadPool().submit(getFromDataNodeCallable);
}
private ByteBuffer getFirstToComplete(ArrayList<Future<ByteBuffer>> futures,
CountDownLatch latch) throws ExecutionException, InterruptedException {
latch.await();
for (Future<ByteBuffer> future : futures) {
if (future.isDone()) {
try {
return future.get();
} catch (ExecutionException e) {
// already logged in the Callable
futures.remove(future);
throw e;
}
}
}
throw new InterruptedException("latch has counted down to zero but no"
+ "result available yet, for safety try to request another one from"
+ "outside loop, this should be rare");
}
private void cancelAll(List<Future<ByteBuffer>> futures) {
for (Future<ByteBuffer> future : futures) {
// Unfortunately, hdfs reads do not take kindly to interruption.
// Threads return a variety of interrupted-type exceptions but
// also complaints about invalid pbs -- likely because read
// is interrupted before gets whole pb. Also verbose WARN
// logging. So, for now, do not interrupt running read.
future.cancel(false);
}
}
@ -1070,8 +1291,13 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
long targetStart = position - blk.getStartOffset();
long bytesToRead = Math.min(remaining, blk.getBlockSize() - targetStart);
try {
fetchBlockByteRange(blk, targetStart,
targetStart + bytesToRead - 1, buffer, offset, corruptedBlockMap);
if (dfsClient.isHedgedReadsEnabled()) {
hedgedFetchBlockByteRange(blk, targetStart, targetStart + bytesToRead
- 1, buffer, offset, corruptedBlockMap);
} else {
fetchBlockByteRange(blk, targetStart, targetStart + bytesToRead - 1,
buffer, offset, corruptedBlockMap);
}
} finally {
// Check and report if any block replicas are corrupted.
// BlockMissingException may be caught if all block replicas are
@ -1265,12 +1491,13 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
* Pick the best node from which to stream the data.
* Entries in <i>nodes</i> are already in the priority order
*/
static DatanodeInfo bestNode(DatanodeInfo nodes[],
AbstractMap<DatanodeInfo, DatanodeInfo> deadNodes)
throws IOException {
if (nodes != null) {
static DatanodeInfo bestNode(DatanodeInfo nodes[],
AbstractMap<DatanodeInfo, DatanodeInfo> deadNodes,
Collection<DatanodeInfo> ignoredNodes) throws IOException {
if (nodes != null) {
for (int i = 0; i < nodes.length; i++) {
if (!deadNodes.containsKey(nodes[i])) {
if (!deadNodes.containsKey(nodes[i])
&& (ignoredNodes == null || !ignoredNodes.contains(nodes[i]))) {
return nodes[i];
}
}

View File

@ -20,9 +20,14 @@ package org.apache.hadoop.hdfs;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.io.DataOutputStream;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Random;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.conf.Configuration;
@ -33,6 +38,9 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
import org.apache.log4j.Level;
import org.junit.Test;
import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
/**
* This class tests the DFS positional read functionality in a single node
@ -44,9 +52,10 @@ public class TestPread {
boolean simulatedStorage = false;
private void writeFile(FileSystem fileSys, Path name) throws IOException {
int replication = 3;// We need > 1 blocks to test out the hedged reads.
// test empty file open and read
DFSTestUtil.createFile(fileSys, name, 12 * blockSize, 0,
blockSize, (short) 1, seed);
blockSize, (short)replication, seed);
FSDataInputStream in = fileSys.open(name);
byte[] buffer = new byte[12 * blockSize];
in.readFully(0, buffer, 0, 0);
@ -191,26 +200,128 @@ public class TestPread {
assertTrue(fileSys.delete(name, true));
assertTrue(!fileSys.exists(name));
}
private Callable<Void> getPReadFileCallable(final FileSystem fileSys,
final Path file) {
return new Callable<Void>() {
public Void call() throws IOException {
pReadFile(fileSys, file);
return null;
}
};
}
/**
* Tests positional read in DFS.
*/
@Test
public void testPreadDFS() throws IOException {
dfsPreadTest(false, true); //normal pread
dfsPreadTest(true, true); //trigger read code path without transferTo.
Configuration conf = new Configuration();
dfsPreadTest(conf, false, true); // normal pread
dfsPreadTest(conf, true, true); // trigger read code path without
// transferTo.
}
@Test
public void testPreadDFSNoChecksum() throws IOException {
Configuration conf = new Configuration();
((Log4JLogger)DataTransferProtocol.LOG).getLogger().setLevel(Level.ALL);
dfsPreadTest(false, false);
dfsPreadTest(true, false);
dfsPreadTest(conf, false, false);
dfsPreadTest(conf, true, false);
}
private void dfsPreadTest(boolean disableTransferTo, boolean verifyChecksum)
/**
* Tests positional read in DFS, with hedged reads enabled.
*/
@Test
public void testHedgedPreadDFSBasic() throws IOException {
Configuration conf = new Configuration();
conf.setInt(DFSConfigKeys.DFS_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE, 5);
conf.setLong(DFSConfigKeys.DFS_DFSCLIENT_HEDGED_READ_THRESHOLD_MILLIS, 100);
dfsPreadTest(conf, false, true); // normal pread
dfsPreadTest(conf, true, true); // trigger read code path without
// transferTo.
}
@Test
public void testMaxOutHedgedReadPool() throws IOException,
InterruptedException, ExecutionException {
Configuration conf = new Configuration();
int numHedgedReadPoolThreads = 5;
final int initialHedgedReadTimeoutMillis = 500;
final int fixedSleepIntervalMillis = 50;
conf.setInt(DFSConfigKeys.DFS_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE,
numHedgedReadPoolThreads);
conf.setLong(DFSConfigKeys.DFS_DFSCLIENT_HEDGED_READ_THRESHOLD_MILLIS,
initialHedgedReadTimeoutMillis);
// Set up the InjectionHandler
DFSClientFaultInjector.instance = Mockito
.mock(DFSClientFaultInjector.class);
DFSClientFaultInjector injector = DFSClientFaultInjector.instance;
// make preads sleep for 50ms
Mockito.doAnswer(new Answer<Void>() {
@Override
public Void answer(InvocationOnMock invocation) throws Throwable {
Thread.sleep(fixedSleepIntervalMillis);
return null;
}
}).when(injector).startFetchFromDatanode();
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3)
.format(true).build();
DistributedFileSystem fileSys = cluster.getFileSystem();
DFSClient dfsClient = fileSys.getClient();
DFSHedgedReadMetrics metrics = dfsClient.getHedgedReadMetrics();
try {
Path file1 = new Path("hedgedReadMaxOut.dat");
writeFile(fileSys, file1);
// Basic test. Reads complete within timeout. Assert that there were no
// hedged reads.
pReadFile(fileSys, file1);
// assert that there were no hedged reads. 50ms + delta < 500ms
assertTrue(metrics.getHedgedReadOps() == 0);
assertTrue(metrics.getHedgedReadOpsInCurThread() == 0);
/*
* Reads take longer than timeout. But, only one thread reading. Assert
* that there were hedged reads. But, none of the reads had to run in the
* current thread.
*/
dfsClient.setHedgedReadTimeout(50); // 50ms
pReadFile(fileSys, file1);
// assert that there were hedged reads
assertTrue(metrics.getHedgedReadOps() > 0);
assertTrue(metrics.getHedgedReadOpsInCurThread() == 0);
/*
* Multiple threads reading. Reads take longer than timeout. Assert that
* there were hedged reads. And that reads had to run in the current
* thread.
*/
int factor = 10;
int numHedgedReads = numHedgedReadPoolThreads * factor;
long initialReadOpsValue = metrics.getHedgedReadOps();
ExecutorService executor = Executors.newFixedThreadPool(numHedgedReads);
ArrayList<Future<Void>> futures = new ArrayList<Future<Void>>();
for (int i = 0; i < numHedgedReads; i++) {
futures.add(executor.submit(getPReadFileCallable(fileSys, file1)));
}
for (int i = 0; i < numHedgedReads; i++) {
futures.get(i).get();
}
assertTrue(metrics.getHedgedReadOps() > initialReadOpsValue);
assertTrue(metrics.getHedgedReadOpsInCurThread() > 0);
cleanupFile(fileSys, file1);
executor.shutdown();
} finally {
fileSys.close();
cluster.shutdown();
Mockito.reset(injector);
}
}
private void dfsPreadTest(Configuration conf, boolean disableTransferTo, boolean verifyChecksum)
throws IOException {
Configuration conf = new HdfsConfiguration();
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 4096);
conf.setLong(DFSConfigKeys.DFS_CLIENT_READ_PREFETCH_SIZE_KEY, 4096);
if (simulatedStorage) {

View File

@ -28,6 +28,8 @@ Release 2.5.0 - UNRELEASED
YARN-1736. FS: AppSchedulable.assignContainer's priority argument is
redundant. (Naren Koneru via kasha)
YARN-1678. Fair scheduler gabs incessantly about reservations (Sandy Ryza)
OPTIMIZATIONS
BUG FIXES
@ -218,6 +220,15 @@ Release 2.4.0 - UNRELEASED
YARN-1470. Add audience annotations to MiniYARNCluster. (Anubhav Dhoot
via kasha)
YARN-1732. Changed types of related-entities and primary-filters in the
timeline-service to be sets instead of maps. (Billie Rinaldi via vinodkv)
YARN-1687. Renamed user-facing records for the timeline-service to be simply
named after 'timeline' instead of 'apptimeline'. (Zhijie Shen via vinodkv)
YARN-1749. Updated application-history related configs to reflect the latest
reality and to be consistently named. (Zhijie Shen via vinodkv)
OPTIMIZATIONS
BUG FIXES
@ -330,6 +341,13 @@ Release 2.4.0 - UNRELEASED
YARN-1071. Enabled ResourceManager to recover cluster metrics
numDecommissionedNMs after restarting. (Jian He via zjshen)
YARN-1742. Fixed javadoc of configuration parameter
DEFAULT_NM_MIN_HEALTHY_DISKS_FRACTION. (Akira Ajisaka via vinodkv)
YARN-1686. Fixed NodeManager to properly handle any errors during
re-registration after a RESYNC and thus avoid hanging. (Rohith Sharma via
vinodkv)
Release 2.3.1 - UNRELEASED
INCOMPATIBLE CHANGES

View File

@ -16,7 +16,7 @@
* limitations under the License.
*/
package org.apache.hadoop.yarn.api.records.apptimeline;
package org.apache.hadoop.yarn.api.records.timeline;
import java.util.ArrayList;
import java.util.List;
@ -30,18 +30,18 @@ import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
/**
* The class that hosts a list of application timeline entities.
* The class that hosts a list of timeline entities.
*/
@XmlRootElement(name = "entities")
@XmlAccessorType(XmlAccessType.NONE)
@Public
@Unstable
public class ATSEntities {
public class TimelineEntities {
private List<ATSEntity> entities =
new ArrayList<ATSEntity>();
private List<TimelineEntity> entities =
new ArrayList<TimelineEntity>();
public ATSEntities() {
public TimelineEntities() {
}
@ -51,7 +51,7 @@ public class ATSEntities {
* @return a list of entities
*/
@XmlElement(name = "entities")
public List<ATSEntity> getEntities() {
public List<TimelineEntity> getEntities() {
return entities;
}
@ -61,7 +61,7 @@ public class ATSEntities {
* @param entity
* a single entity
*/
public void addEntity(ATSEntity entity) {
public void addEntity(TimelineEntity entity) {
entities.add(entity);
}
@ -71,7 +71,7 @@ public class ATSEntities {
* @param entities
* a list of entities
*/
public void addEntities(List<ATSEntity> entities) {
public void addEntities(List<TimelineEntity> entities) {
this.entities.addAll(entities);
}
@ -81,7 +81,7 @@ public class ATSEntities {
* @param entities
* a list of entities
*/
public void setEntities(List<ATSEntity> entities) {
public void setEntities(List<TimelineEntity> entities) {
this.entities = entities;
}

View File

@ -16,13 +16,15 @@
* limitations under the License.
*/
package org.apache.hadoop.yarn.api.records.apptimeline;
package org.apache.hadoop.yarn.api.records.timeline;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import javax.xml.bind.annotation.XmlAccessType;
import javax.xml.bind.annotation.XmlAccessorType;
@ -34,36 +36,36 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
/**
* <p>
* The class that contains the the meta information of some conceptual entity of
* an application and its related events. The entity can be an application, an
* application attempt, a container or whatever the user-defined object.
* The class that contains the the meta information of some conceptual entity
* and its related events. The entity can be an application, an application
* attempt, a container or whatever the user-defined object.
* </p>
*
* <p>
* Primary filters will be used to index the entities in
* <code>ApplicationTimelineStore</code>, such that users should carefully
* choose the information they want to store as the primary filters. The
* remaining can be stored as other information.
* <code>TimelineStore</code>, such that users should carefully choose the
* information they want to store as the primary filters. The remaining can be
* stored as other information.
* </p>
*/
@XmlRootElement(name = "entity")
@XmlAccessorType(XmlAccessType.NONE)
@Public
@Unstable
public class ATSEntity implements Comparable<ATSEntity> {
public class TimelineEntity implements Comparable<TimelineEntity> {
private String entityType;
private String entityId;
private Long startTime;
private List<ATSEvent> events = new ArrayList<ATSEvent>();
private Map<String, List<String>> relatedEntities =
new HashMap<String, List<String>>();
private Map<String, Object> primaryFilters =
new HashMap<String, Object>();
private List<TimelineEvent> events = new ArrayList<TimelineEvent>();
private Map<String, Set<String>> relatedEntities =
new HashMap<String, Set<String>>();
private Map<String, Set<Object>> primaryFilters =
new HashMap<String, Set<Object>>();
private Map<String, Object> otherInfo =
new HashMap<String, Object>();
public ATSEntity() {
public TimelineEntity() {
}
@ -133,7 +135,7 @@ public class ATSEntity implements Comparable<ATSEntity> {
* @return a list of events related to the entity
*/
@XmlElement(name = "events")
public List<ATSEvent> getEvents() {
public List<TimelineEvent> getEvents() {
return events;
}
@ -143,7 +145,7 @@ public class ATSEntity implements Comparable<ATSEntity> {
* @param event
* a single event related to the entity
*/
public void addEvent(ATSEvent event) {
public void addEvent(TimelineEvent event) {
events.add(event);
}
@ -153,7 +155,7 @@ public class ATSEntity implements Comparable<ATSEntity> {
* @param events
* a list of events related to the entity
*/
public void addEvents(List<ATSEvent> events) {
public void addEvents(List<TimelineEvent> events) {
this.events.addAll(events);
}
@ -163,7 +165,7 @@ public class ATSEntity implements Comparable<ATSEntity> {
* @param events
* events a list of events related to the entity
*/
public void setEvents(List<ATSEvent> events) {
public void setEvents(List<TimelineEvent> events) {
this.events = events;
}
@ -173,7 +175,7 @@ public class ATSEntity implements Comparable<ATSEntity> {
* @return the related entities
*/
@XmlElement(name = "relatedentities")
public Map<String, List<String>> getRelatedEntities() {
public Map<String, Set<String>> getRelatedEntities() {
return relatedEntities;
}
@ -186,9 +188,9 @@ public class ATSEntity implements Comparable<ATSEntity> {
* the entity Id
*/
public void addRelatedEntity(String entityType, String entityId) {
List<String> thisRelatedEntity = relatedEntities.get(entityType);
Set<String> thisRelatedEntity = relatedEntities.get(entityType);
if (thisRelatedEntity == null) {
thisRelatedEntity = new ArrayList<String>();
thisRelatedEntity = new HashSet<String>();
relatedEntities.put(entityType, thisRelatedEntity);
}
thisRelatedEntity.add(entityId);
@ -200,10 +202,9 @@ public class ATSEntity implements Comparable<ATSEntity> {
* @param relatedEntities
* a map of related entities
*/
public void addRelatedEntities(Map<String, List<String>> relatedEntities) {
for (Entry<String, List<String>> relatedEntity :
relatedEntities.entrySet()) {
List<String> thisRelatedEntity =
public void addRelatedEntities(Map<String, Set<String>> relatedEntities) {
for (Entry<String, Set<String>> relatedEntity : relatedEntities.entrySet()) {
Set<String> thisRelatedEntity =
this.relatedEntities.get(relatedEntity.getKey());
if (thisRelatedEntity == null) {
this.relatedEntities.put(
@ -221,7 +222,7 @@ public class ATSEntity implements Comparable<ATSEntity> {
* a map of related entities
*/
public void setRelatedEntities(
Map<String, List<String>> relatedEntities) {
Map<String, Set<String>> relatedEntities) {
this.relatedEntities = relatedEntities;
}
@ -231,7 +232,7 @@ public class ATSEntity implements Comparable<ATSEntity> {
* @return the primary filters
*/
@XmlElement(name = "primaryfilters")
public Map<String, Object> getPrimaryFilters() {
public Map<String, Set<Object>> getPrimaryFilters() {
return primaryFilters;
}
@ -244,7 +245,12 @@ public class ATSEntity implements Comparable<ATSEntity> {
* the primary filter value
*/
public void addPrimaryFilter(String key, Object value) {
primaryFilters.put(key, value);
Set<Object> thisPrimaryFilter = primaryFilters.get(key);
if (thisPrimaryFilter == null) {
thisPrimaryFilter = new HashSet<Object>();
primaryFilters.put(key, thisPrimaryFilter);
}
thisPrimaryFilter.add(value);
}
/**
@ -253,8 +259,17 @@ public class ATSEntity implements Comparable<ATSEntity> {
* @param primaryFilters
* a map of primary filters
*/
public void addPrimaryFilters(Map<String, Object> primaryFilters) {
this.primaryFilters.putAll(primaryFilters);
public void addPrimaryFilters(Map<String, Set<Object>> primaryFilters) {
for (Entry<String, Set<Object>> primaryFilter : primaryFilters.entrySet()) {
Set<Object> thisPrimaryFilter =
this.primaryFilters.get(primaryFilter.getKey());
if (thisPrimaryFilter == null) {
this.primaryFilters.put(
primaryFilter.getKey(), primaryFilter.getValue());
} else {
thisPrimaryFilter.addAll(primaryFilter.getValue());
}
}
}
/**
@ -263,7 +278,7 @@ public class ATSEntity implements Comparable<ATSEntity> {
* @param primaryFilters
* a map of primary filters
*/
public void setPrimaryFilters(Map<String, Object> primaryFilters) {
public void setPrimaryFilters(Map<String, Set<Object>> primaryFilters) {
this.primaryFilters = primaryFilters;
}
@ -339,7 +354,7 @@ public class ATSEntity implements Comparable<ATSEntity> {
return false;
if (getClass() != obj.getClass())
return false;
ATSEntity other = (ATSEntity) obj;
TimelineEntity other = (TimelineEntity) obj;
if (entityId == null) {
if (other.entityId != null)
return false;
@ -379,7 +394,7 @@ public class ATSEntity implements Comparable<ATSEntity> {
}
@Override
public int compareTo(ATSEntity other) {
public int compareTo(TimelineEntity other) {
int comparison = entityType.compareTo(other.entityType);
if (comparison == 0) {
long thisStartTime =

View File

@ -16,7 +16,7 @@
* limitations under the License.
*/
package org.apache.hadoop.yarn.api.records.apptimeline;
package org.apache.hadoop.yarn.api.records.timeline;
import java.util.HashMap;
import java.util.Map;
@ -39,13 +39,13 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
@XmlAccessorType(XmlAccessType.NONE)
@Public
@Unstable
public class ATSEvent implements Comparable<ATSEvent> {
public class TimelineEvent implements Comparable<TimelineEvent> {
private long timestamp;
private String eventType;
private Map<String, Object> eventInfo = new HashMap<String, Object>();
public ATSEvent() {
public TimelineEvent() {
}
/**
@ -132,7 +132,7 @@ public class ATSEvent implements Comparable<ATSEvent> {
}
@Override
public int compareTo(ATSEvent other) {
public int compareTo(TimelineEvent other) {
if (timestamp > other.timestamp) {
return -1;
} else if (timestamp < other.timestamp) {
@ -149,14 +149,14 @@ public class ATSEvent implements Comparable<ATSEvent> {
if (o == null || getClass() != o.getClass())
return false;
ATSEvent atsEvent = (ATSEvent) o;
TimelineEvent event = (TimelineEvent) o;
if (timestamp != atsEvent.timestamp)
if (timestamp != event.timestamp)
return false;
if (!eventType.equals(atsEvent.eventType))
if (!eventType.equals(event.eventType))
return false;
if (eventInfo != null ? !eventInfo.equals(atsEvent.eventInfo) :
atsEvent.eventInfo != null)
if (eventInfo != null ? !eventInfo.equals(event.eventInfo) :
event.eventInfo != null)
return false;
return true;

View File

@ -16,7 +16,7 @@
* limitations under the License.
*/
package org.apache.hadoop.yarn.api.records.apptimeline;
package org.apache.hadoop.yarn.api.records.timeline;
import java.util.ArrayList;
import java.util.List;
@ -37,52 +37,52 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
@XmlAccessorType(XmlAccessType.NONE)
@Public
@Unstable
public class ATSEvents {
public class TimelineEvents {
private List<ATSEventsOfOneEntity> allEvents =
new ArrayList<ATSEventsOfOneEntity>();
private List<EventsOfOneEntity> allEvents =
new ArrayList<EventsOfOneEntity>();
public ATSEvents() {
public TimelineEvents() {
}
/**
* Get a list of {@link ATSEventsOfOneEntity} instances
* Get a list of {@link EventsOfOneEntity} instances
*
* @return a list of {@link ATSEventsOfOneEntity} instances
* @return a list of {@link EventsOfOneEntity} instances
*/
@XmlElement(name = "events")
public List<ATSEventsOfOneEntity> getAllEvents() {
public List<EventsOfOneEntity> getAllEvents() {
return allEvents;
}
/**
* Add a single {@link ATSEventsOfOneEntity} instance into the existing list
* Add a single {@link EventsOfOneEntity} instance into the existing list
*
* @param eventsOfOneEntity
* a single {@link ATSEventsOfOneEntity} instance
* a single {@link EventsOfOneEntity} instance
*/
public void addEvent(ATSEventsOfOneEntity eventsOfOneEntity) {
public void addEvent(EventsOfOneEntity eventsOfOneEntity) {
allEvents.add(eventsOfOneEntity);
}
/**
* Add a list of {@link ATSEventsOfOneEntity} instances into the existing list
* Add a list of {@link EventsOfOneEntity} instances into the existing list
*
* @param allEvents
* a list of {@link ATSEventsOfOneEntity} instances
* a list of {@link EventsOfOneEntity} instances
*/
public void addEvents(List<ATSEventsOfOneEntity> allEvents) {
public void addEvents(List<EventsOfOneEntity> allEvents) {
this.allEvents.addAll(allEvents);
}
/**
* Set the list to the given list of {@link ATSEventsOfOneEntity} instances
* Set the list to the given list of {@link EventsOfOneEntity} instances
*
* @param allEvents
* a list of {@link ATSEventsOfOneEntity} instances
* a list of {@link EventsOfOneEntity} instances
*/
public void setEvents(List<ATSEventsOfOneEntity> allEvents) {
public void setEvents(List<EventsOfOneEntity> allEvents) {
this.allEvents.clear();
this.allEvents.addAll(allEvents);
}
@ -94,13 +94,13 @@ public class ATSEvents {
@XmlAccessorType(XmlAccessType.NONE)
@Public
@Unstable
public static class ATSEventsOfOneEntity {
public static class EventsOfOneEntity {
private String entityId;
private String entityType;
private List<ATSEvent> events = new ArrayList<ATSEvent>();
private List<TimelineEvent> events = new ArrayList<TimelineEvent>();
public ATSEventsOfOneEntity() {
public EventsOfOneEntity() {
}
@ -150,7 +150,7 @@ public class ATSEvents {
* @return a list of events
*/
@XmlElement(name = "events")
public List<ATSEvent> getEvents() {
public List<TimelineEvent> getEvents() {
return events;
}
@ -160,7 +160,7 @@ public class ATSEvents {
* @param event
* a single event
*/
public void addEvent(ATSEvent event) {
public void addEvent(TimelineEvent event) {
events.add(event);
}
@ -170,7 +170,7 @@ public class ATSEvents {
* @param events
* a list of events
*/
public void addEvents(List<ATSEvent> events) {
public void addEvents(List<TimelineEvent> events) {
this.events.addAll(events);
}
@ -180,7 +180,7 @@ public class ATSEvents {
* @param events
* a list of events
*/
public void setEvents(List<ATSEvent> events) {
public void setEvents(List<TimelineEvent> events) {
this.events = events;
}

View File

@ -15,7 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.api.records.apptimeline;
package org.apache.hadoop.yarn.api.records.timeline;
import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
@ -28,60 +28,60 @@ import java.util.ArrayList;
import java.util.List;
/**
* A class that holds a list of put errors. This is the response returned
* when a list of {@link ATSEntity} objects is added to the application
* timeline. If there are errors in storing individual entity objects,
* they will be indicated in the list of errors.
* A class that holds a list of put errors. This is the response returned when a
* list of {@link TimelineEntity} objects is added to the timeline. If there are errors
* in storing individual entity objects, they will be indicated in the list of
* errors.
*/
@XmlRootElement(name = "errors")
@XmlRootElement(name = "response")
@XmlAccessorType(XmlAccessType.NONE)
@Public
@Unstable
public class ATSPutErrors {
public class TimelinePutResponse {
private List<ATSPutError> errors = new ArrayList<ATSPutError>();
private List<TimelinePutError> errors = new ArrayList<TimelinePutError>();
public ATSPutErrors() {
public TimelinePutResponse() {
}
/**
* Get a list of {@link ATSPutError} instances
*
* @return a list of {@link ATSPutError} instances
* Get a list of {@link TimelinePutError} instances
*
* @return a list of {@link TimelinePutError} instances
*/
@XmlElement(name = "errors")
public List<ATSPutError> getErrors() {
public List<TimelinePutError> getErrors() {
return errors;
}
/**
* Add a single {@link ATSPutError} instance into the existing list
*
* Add a single {@link TimelinePutError} instance into the existing list
*
* @param error
* a single {@link ATSPutError} instance
* a single {@link TimelinePutError} instance
*/
public void addError(ATSPutError error) {
public void addError(TimelinePutError error) {
errors.add(error);
}
/**
* Add a list of {@link ATSPutError} instances into the existing list
*
* Add a list of {@link TimelinePutError} instances into the existing list
*
* @param errors
* a list of {@link ATSPutError} instances
* a list of {@link TimelinePutError} instances
*/
public void addErrors(List<ATSPutError> errors) {
public void addErrors(List<TimelinePutError> errors) {
this.errors.addAll(errors);
}
/**
* Set the list to the given list of {@link ATSPutError} instances
*
* Set the list to the given list of {@link TimelinePutError} instances
*
* @param errors
* a list of {@link ATSPutError} instances
* a list of {@link TimelinePutError} instances
*/
public void setErrors(List<ATSPutError> errors) {
public void setErrors(List<TimelinePutError> errors) {
this.errors.clear();
this.errors.addAll(errors);
}
@ -93,11 +93,12 @@ public class ATSPutErrors {
@XmlAccessorType(XmlAccessType.NONE)
@Public
@Unstable
public static class ATSPutError {
public static class TimelinePutError {
/**
* Error code returned when no start time can be found when putting an
* entity. This occurs when the entity does not already exist in the
* store and it is put with no start time or events specified.
* entity. This occurs when the entity does not already exist in the store
* and it is put with no start time or events specified.
*/
public static final int NO_START_TIME = 1;
/**
@ -112,7 +113,7 @@ public class ATSPutErrors {
/**
* Get the entity Id
*
*
* @return the entity Id
*/
@XmlElement(name = "entity")
@ -122,7 +123,7 @@ public class ATSPutErrors {
/**
* Set the entity Id
*
*
* @param entityId
* the entity Id
*/
@ -132,7 +133,7 @@ public class ATSPutErrors {
/**
* Get the entity type
*
*
* @return the entity type
*/
@XmlElement(name = "entitytype")
@ -142,7 +143,7 @@ public class ATSPutErrors {
/**
* Set the entity type
*
*
* @param entityType
* the entity type
*/
@ -152,7 +153,7 @@ public class ATSPutErrors {
/**
* Get the error code
*
*
* @return an error code
*/
@XmlElement(name = "errorcode")
@ -162,7 +163,7 @@ public class ATSPutErrors {
/**
* Set the error code to the given error code
*
*
* @param errorCode
* an error code
*/

View File

@ -16,6 +16,6 @@
* limitations under the License.
*/
@InterfaceAudience.Public
package org.apache.hadoop.yarn.api.records.apptimeline;
package org.apache.hadoop.yarn.api.records.timeline;
import org.apache.hadoop.classification.InterfaceAudience;

View File

@ -300,11 +300,6 @@ public class YarnConfiguration extends Configuration {
public static final int DEFAULT_RM_HISTORY_WRITER_MULTI_THREADED_DISPATCHER_POOL_SIZE =
10;
/** The implementation class of ApplicationHistoryStore, which is to be used
* by RMApplicationHistoryWriter. */
public static final String RM_HISTORY_WRITER_CLASS = RM_PREFIX
+ "history-writer.class";
//Delegation token related keys
public static final String DELEGATION_KEY_UPDATE_INTERVAL_KEY =
RM_PREFIX + "delegation.key.update-interval";
@ -741,7 +736,7 @@ public class YarnConfiguration extends Configuration {
public static final String NM_MIN_HEALTHY_DISKS_FRACTION =
NM_PREFIX + "disk-health-checker.min-healthy-disks";
/**
* By default, at least 5% of disks are to be healthy to say that the node
* By default, at least 25% of disks are to be healthy to say that the node
* is healthy in terms of disks.
*/
public static final float DEFAULT_NM_MIN_HEALTHY_DISKS_FRACTION
@ -994,74 +989,91 @@ public class YarnConfiguration extends Configuration {
YARN_PREFIX + "app.container.log.backups";
////////////////////////////////
// AHS Configs
// Timeline Service Configs
////////////////////////////////
public static final String AHS_PREFIX = YARN_PREFIX + "ahs.";
public static final String TIMELINE_SERVICE_PREFIX =
YARN_PREFIX + "timeline-service.";
/** The setting that controls whether history-service is enabled or not.. */
public static final String YARN_HISTORY_SERVICE_ENABLED = AHS_PREFIX
+ "enabled";
public static final boolean DEFAULT_YARN_HISTORY_SERVICE_ENABLED = false;
// mark app-history related configs @Private as application history is going
// to be integrated into the timeline service
@Private
public static final String APPLICATION_HISTORY_PREFIX =
TIMELINE_SERVICE_PREFIX + "generic-application-history.";
/**
* The setting that controls whether application history service is
* enabled or not.
*/
@Private
public static final String APPLICATION_HISTORY_ENABLED =
APPLICATION_HISTORY_PREFIX + "enabled";
@Private
public static final boolean DEFAULT_APPLICATION_HISTORY_ENABLED = false;
/** Application history store class */
@Private
public static final String APPLICATION_HISTORY_STORE =
APPLICATION_HISTORY_PREFIX + "store-class";
/** URI for FileSystemApplicationHistoryStore */
public static final String FS_HISTORY_STORE_URI = AHS_PREFIX + "fs-history-store.uri";
@Private
public static final String FS_APPLICATION_HISTORY_STORE_URI =
APPLICATION_HISTORY_PREFIX + "fs-history-store.uri";
/** T-file compression types used to compress history data.*/
public static final String FS_HISTORY_STORE_COMPRESSION_TYPE = AHS_PREFIX + "fs-history-store.compression-type";
public static final String DEFAULT_FS_HISTORY_STORE_COMPRESSION_TYPE = "none";
@Private
public static final String FS_APPLICATION_HISTORY_STORE_COMPRESSION_TYPE =
APPLICATION_HISTORY_PREFIX + "fs-history-store.compression-type";
@Private
public static final String DEFAULT_FS_APPLICATION_HISTORY_STORE_COMPRESSION_TYPE =
"none";
/** AHS store class */
public static final String AHS_STORE = AHS_PREFIX + "store.class";
/** host:port address for timeline service RPC APIs. */
public static final String TIMELINE_SERVICE_ADDRESS =
TIMELINE_SERVICE_PREFIX + "address";
public static final int DEFAULT_TIMELINE_SERVICE_PORT = 10200;
public static final String DEFAULT_TIMELINE_SERVICE_ADDRESS = "0.0.0.0:"
+ DEFAULT_TIMELINE_SERVICE_PORT;
/** host:port address for Application History Server API. */
public static final String AHS_ADDRESS = AHS_PREFIX + "address";
public static final int DEFAULT_AHS_PORT = 10200;
public static final String DEFAULT_AHS_ADDRESS = "0.0.0.0:"
+ DEFAULT_AHS_PORT;
/** The number of threads to handle client API requests. */
public static final String AHS_CLIENT_THREAD_COUNT = AHS_PREFIX
+ "client.thread-count";
public static final int DEFAULT_AHS_CLIENT_THREAD_COUNT = 10;
/** The number of threads to handle client RPC API requests. */
public static final String TIMELINE_SERVICE_HANDLER_THREAD_COUNT =
TIMELINE_SERVICE_PREFIX + "handler-thread-count";
public static final int DEFAULT_TIMELINE_SERVICE_CLIENT_THREAD_COUNT = 10;
/** The address of the AHS web application.*/
public static final String AHS_WEBAPP_ADDRESS = AHS_PREFIX
+ "webapp.address";
/** The address of the timeline service web application.*/
public static final String TIMELINE_SERVICE_WEBAPP_ADDRESS =
TIMELINE_SERVICE_PREFIX + "webapp.address";
public static final int DEFAULT_AHS_WEBAPP_PORT = 8188;
public static final String DEFAULT_AHS_WEBAPP_ADDRESS = "0.0.0.0:"
+ DEFAULT_AHS_WEBAPP_PORT;
public static final int DEFAULT_TIMELINE_SERVICE_WEBAPP_PORT = 8188;
public static final String DEFAULT_TIMELINE_SERVICE_WEBAPP_ADDRESS =
"0.0.0.0:" + DEFAULT_TIMELINE_SERVICE_WEBAPP_PORT;
/** The https address of the AHS web application.*/
public static final String AHS_WEBAPP_HTTPS_ADDRESS = AHS_PREFIX
+ "webapp.https.address";
/** The https address of the timeline service web application.*/
public static final String TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS =
TIMELINE_SERVICE_PREFIX + "webapp.https.address";
public static final int DEFAULT_AHS_WEBAPP_HTTPS_PORT = 8190;
public static final String DEFAULT_AHS_WEBAPP_HTTPS_ADDRESS = "0.0.0.0:"
+ DEFAULT_AHS_WEBAPP_HTTPS_PORT;
public static final int DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_PORT = 8190;
public static final String DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS =
"0.0.0.0:" + DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_PORT;
/**The kerberos principal to be used for spnego filter for AHS.*/
public static final String AHS_WEBAPP_SPNEGO_USER_NAME_KEY =
AHS_PREFIX + "webapp.spnego-principal";
/**The kerberos principal to be used for spnego filter for timeline service.*/
public static final String TIMELINE_SERVICE_WEBAPP_SPNEGO_USER_NAME_KEY =
TIMELINE_SERVICE_PREFIX + "webapp.spnego-principal";
/**The kerberos keytab to be used for spnego filter for AHS.*/
public static final String AHS_WEBAPP_SPNEGO_KEYTAB_FILE_KEY =
AHS_PREFIX + "webapp.spnego-keytab-file";
/**The kerberos keytab to be used for spnego filter for timeline service.*/
public static final String TIMELINE_SERVICE_WEBAPP_SPNEGO_KEYTAB_FILE_KEY =
TIMELINE_SERVICE_PREFIX + "webapp.spnego-keytab-file";
////////////////////////////////
// ATS Configs
////////////////////////////////
/** Timeline service store class */
public static final String TIMELINE_SERVICE_STORE =
TIMELINE_SERVICE_PREFIX + "store-class";
public static final String ATS_PREFIX = YARN_PREFIX + "ats.";
/** ATS store class */
public static final String ATS_STORE = ATS_PREFIX + "store.class";
/** ATS leveldb path */
public static final String ATS_LEVELDB_PATH_PROPERTY =
ATS_PREFIX + "leveldb-apptimeline-store.path";
/** Timeline service leveldb path */
public static final String TIMELINE_SERVICE_LEVELDB_PATH =
TIMELINE_SERVICE_PREFIX + "leveldb-timeline-store.path";
////////////////////////////////
// Other Configs

View File

@ -24,8 +24,8 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSEntity;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSPutErrors;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
import org.apache.hadoop.yarn.client.api.impl.TimelineClientImpl;
import org.apache.hadoop.yarn.exceptions.YarnException;
@ -33,7 +33,7 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
* A client library that can be used to post some information in terms of a
* number of conceptual entities.
*
* @See ATSEntity
* @See Entity
*/
@Public
@Unstable
@ -52,19 +52,19 @@ public abstract class TimelineClient extends AbstractService {
/**
* <p>
* Post the information of a number of conceptual entities of an application
* to the timeline server. It is a blocking API. The method will not return
* until it gets the response from the timeline server.
* Send the information of a number of conceptual entities to the timeline
* server. It is a blocking API. The method will not return until it gets the
* response from the timeline server.
* </p>
*
* @param entities
* the collection of {@link ATSEntity}
* @return the error information if the post entities are not correctly stored
* the collection of {@link TimelineEntity}
* @return the error information if the sent entities are not correctly stored
* @throws IOException
* @throws YarnException
*/
@Public
public abstract ATSPutErrors postEntities(
ATSEntity... entities) throws IOException, YarnException;
public abstract TimelinePutResponse putEntities(
TimelineEntity... entities) throws IOException, YarnException;
}

View File

@ -63,9 +63,9 @@ public class AHSClientImpl extends AHSClient {
}
private static InetSocketAddress getAHSAddress(Configuration conf) {
return conf.getSocketAddr(YarnConfiguration.AHS_ADDRESS,
YarnConfiguration.DEFAULT_AHS_ADDRESS,
YarnConfiguration.DEFAULT_AHS_PORT);
return conf.getSocketAddr(YarnConfiguration.TIMELINE_SERVICE_ADDRESS,
YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ADDRESS,
YarnConfiguration.DEFAULT_TIMELINE_SERVICE_PORT);
}
@Override

View File

@ -29,9 +29,9 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSEntities;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSEntity;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSPutErrors;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
import org.apache.hadoop.yarn.client.api.TimelineClient;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.YarnException;
@ -50,7 +50,7 @@ import com.sun.jersey.api.client.config.DefaultClientConfig;
public class TimelineClientImpl extends TimelineClient {
private static final Log LOG = LogFactory.getLog(TimelineClientImpl.class);
private static final String RESOURCE_URI_STR = "/ws/v1/apptimeline/";
private static final String RESOURCE_URI_STR = "/ws/v1/timeline/";
private static final Joiner JOINER = Joiner.on("");
private Client client;
@ -67,21 +67,21 @@ public class TimelineClientImpl extends TimelineClient {
if (YarnConfiguration.useHttps(conf)) {
resURI = URI
.create(JOINER.join("https://", conf.get(
YarnConfiguration.AHS_WEBAPP_HTTPS_ADDRESS,
YarnConfiguration.DEFAULT_AHS_WEBAPP_HTTPS_ADDRESS),
YarnConfiguration.TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS,
YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS),
RESOURCE_URI_STR));
} else {
resURI = URI.create(JOINER.join("http://", conf.get(
YarnConfiguration.AHS_WEBAPP_ADDRESS,
YarnConfiguration.DEFAULT_AHS_WEBAPP_ADDRESS), RESOURCE_URI_STR));
YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_ADDRESS), RESOURCE_URI_STR));
}
super.serviceInit(conf);
}
@Override
public ATSPutErrors postEntities(
ATSEntity... entities) throws IOException, YarnException {
ATSEntities entitiesContainer = new ATSEntities();
public TimelinePutResponse putEntities(
TimelineEntity... entities) throws IOException, YarnException {
TimelineEntities entitiesContainer = new TimelineEntities();
entitiesContainer.addEntities(Arrays.asList(entities));
ClientResponse resp = doPostingEntities(entitiesContainer);
if (resp.getClientResponseStatus() != ClientResponse.Status.OK) {
@ -95,12 +95,12 @@ public class TimelineClientImpl extends TimelineClient {
}
throw new YarnException(msg);
}
return resp.getEntity(ATSPutErrors.class);
return resp.getEntity(TimelinePutResponse.class);
}
@Private
@VisibleForTesting
public ClientResponse doPostingEntities(ATSEntities entities) {
public ClientResponse doPostingEntities(TimelineEntities entities) {
WebResource webResource = client.resource(resURI);
return webResource.accept(MediaType.APPLICATION_JSON)
.type(MediaType.APPLICATION_JSON)

View File

@ -114,8 +114,8 @@ public class YarnClientImpl extends YarnClient {
YarnConfiguration.DEFAULT_YARN_CLIENT_APPLICATION_CLIENT_PROTOCOL_POLL_INTERVAL_MS);
}
if (conf.getBoolean(YarnConfiguration.YARN_HISTORY_SERVICE_ENABLED,
YarnConfiguration.DEFAULT_YARN_HISTORY_SERVICE_ENABLED)) {
if (conf.getBoolean(YarnConfiguration.APPLICATION_HISTORY_ENABLED,
YarnConfiguration.DEFAULT_APPLICATION_HISTORY_ENABLED)) {
historyServiceEnabled = true;
historyClient = AHSClientImpl.createAHSClient();
historyClient.init(getConfig());

View File

@ -25,10 +25,10 @@ import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.when;
import junit.framework.Assert;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSEntities;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSEntity;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSEvent;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSPutErrors;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
import org.apache.hadoop.yarn.client.api.TimelineClient;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.YarnException;
@ -58,8 +58,8 @@ public class TestTimelineClient {
public void testPostEntities() throws Exception {
mockClientResponse(ClientResponse.Status.OK, false);
try {
ATSPutErrors errors = client.postEntities(generateATSEntity());
Assert.assertEquals(0, errors.getErrors().size());
TimelinePutResponse response = client.putEntities(generateEntity());
Assert.assertEquals(0, response.getErrors().size());
} catch (YarnException e) {
Assert.fail("Exception is not expected");
}
@ -69,14 +69,14 @@ public class TestTimelineClient {
public void testPostEntitiesWithError() throws Exception {
mockClientResponse(ClientResponse.Status.OK, true);
try {
ATSPutErrors errors = client.postEntities(generateATSEntity());
Assert.assertEquals(1, errors.getErrors().size());
Assert.assertEquals("test entity id", errors.getErrors().get(0)
TimelinePutResponse response = client.putEntities(generateEntity());
Assert.assertEquals(1, response.getErrors().size());
Assert.assertEquals("test entity id", response.getErrors().get(0)
.getEntityId());
Assert.assertEquals("test entity type", errors.getErrors().get(0)
Assert.assertEquals("test entity type", response.getErrors().get(0)
.getEntityType());
Assert.assertEquals(ATSPutErrors.ATSPutError.IO_EXCEPTION,
errors.getErrors().get(0).getErrorCode());
Assert.assertEquals(TimelinePutResponse.TimelinePutError.IO_EXCEPTION,
response.getErrors().get(0).getErrorCode());
} catch (YarnException e) {
Assert.fail("Exception is not expected");
}
@ -86,7 +86,7 @@ public class TestTimelineClient {
public void testPostEntitiesNoResponse() throws Exception {
mockClientResponse(ClientResponse.Status.INTERNAL_SERVER_ERROR, false);
try {
client.postEntities(generateATSEntity());
client.putEntities(generateEntity());
Assert.fail("Exception is expected");
} catch (YarnException e) {
Assert.assertTrue(e.getMessage().contains(
@ -98,27 +98,28 @@ public class TestTimelineClient {
boolean hasError) {
ClientResponse response = mock(ClientResponse.class);
doReturn(response).when(client)
.doPostingEntities(any(ATSEntities.class));
.doPostingEntities(any(TimelineEntities.class));
when(response.getClientResponseStatus()).thenReturn(status);
ATSPutErrors.ATSPutError error = new ATSPutErrors.ATSPutError();
TimelinePutResponse.TimelinePutError error =
new TimelinePutResponse.TimelinePutError();
error.setEntityId("test entity id");
error.setEntityType("test entity type");
error.setErrorCode(ATSPutErrors.ATSPutError.IO_EXCEPTION);
ATSPutErrors errors = new ATSPutErrors();
error.setErrorCode(TimelinePutResponse.TimelinePutError.IO_EXCEPTION);
TimelinePutResponse putResponse = new TimelinePutResponse();
if (hasError) {
errors.addError(error);
putResponse.addError(error);
}
when(response.getEntity(ATSPutErrors.class)).thenReturn(errors);
when(response.getEntity(TimelinePutResponse.class)).thenReturn(putResponse);
return response;
}
private static ATSEntity generateATSEntity() {
ATSEntity entity = new ATSEntity();
private static TimelineEntity generateEntity() {
TimelineEntity entity = new TimelineEntity();
entity.setEntityId("entity id");
entity.setEntityType("entity type");
entity.setStartTime(System.currentTimeMillis());
for (int i = 0; i < 2; ++i) {
ATSEvent event = new ATSEvent();
TimelineEvent event = new TimelineEvent();
event.setTimestamp(System.currentTimeMillis());
event.setEventType("test event type " + i);
event.addEventInfo("key1", "val1");

View File

@ -16,7 +16,7 @@
* limitations under the License.
*/
package org.apache.hadoop.yarn.util;
package org.apache.hadoop.yarn.util.timeline;
import java.io.IOException;

View File

@ -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.
*/
@InterfaceAudience.Public
package org.apache.hadoop.yarn.util.timeline;
import org.apache.hadoop.classification.InterfaceAudience;

View File

@ -142,11 +142,11 @@ public class WebAppUtils {
public static String getAHSWebAppURLWithoutScheme(Configuration conf) {
if (YarnConfiguration.useHttps(conf)) {
return conf.get(YarnConfiguration.AHS_WEBAPP_HTTPS_ADDRESS,
YarnConfiguration.DEFAULT_AHS_WEBAPP_HTTPS_ADDRESS);
return conf.get(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS,
YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS);
} else {
return conf.get(YarnConfiguration.AHS_WEBAPP_ADDRESS,
YarnConfiguration.DEFAULT_AHS_WEBAPP_ADDRESS);
return conf.get(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_ADDRESS);
}
}

View File

@ -572,30 +572,12 @@
<value>org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy</value>
</property>
<property>
<description>Indicate to ResourceManager as well as clients whether
history-service is enabled or not. If enabled, ResourceManager starts
recording historical data that ApplicationHistory service can consume.
Similarly, clients can redirect to the history service when applications
finish if this is enabled.</description>
<name>yarn.ahs.enabled</name>
<value>false</value>
</property>
<property>
<description>Number of worker threads that write the history data.</description>
<name>yarn.resourcemanager.history-writer.multi-threaded-dispatcher.pool-size</name>
<value>10</value>
</property>
<property>
<description>The implementation class of ApplicationHistoryStore, which is
to be used by RMApplicationHistoryWriter.
</description>
<name>yarn.resourcemanager.history-writer.class</name>
<value>org.apache.hadoop.yarn.server.applicationhistoryservice.NullApplicationHistoryStore</value>
</property>
<property>
<description>The class to use as the configuration provider.
If org.apache.hadoop.yarn.LocalConfigurationProvider is used,
@ -1085,75 +1067,83 @@
<value></value>
</property>
<!-- Application History Service's Configuration-->
<!-- Timeline Service's Configuration-->
<property>
<description>The hostname of the AHS.</description>
<name>yarn.ahs.hostname</name>
<description>The hostname of the timeline service web application.</description>
<name>yarn.timeline-service.hostname</name>
<value>0.0.0.0</value>
</property>
<property>
<description>The http address of the AHS web application.</description>
<name>yarn.ahs.webapp.address</name>
<value>${yarn.ahs.hostname}:8188</value>
<description>This is default address for the timeline server to start the
RPC server.</description>
<name>yarn.timeline-service.address</name>
<value>0.0.0.0:10200</value>
</property>
<property>
<description>The https adddress of the AHS web application.</description>
<name>yarn.ahs.webapp.https.address</name>
<value>${yarn.ahs.hostname}:8190</value>
<description>The http address of the timeline service web application.</description>
<name>yarn.timeline-service.webapp.address</name>
<value>${yarn.timeline-service.hostname}:8188</value>
</property>
<property>
<description>The https adddress of the timeline service web application.</description>
<name>yarn.timeline-service.webapp.https.address</name>
<value>${yarn.timeline-service.hostname}:8190</value>
</property>
<property>
<description>Store class name for timeline store</description>
<name>yarn.timeline-service.store-class</name>
<value>org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.LeveldbTimelineStore</value>
</property>
<property>
<description>Store file name for leveldb timeline store</description>
<name>yarn.timeline-service.leveldb-timeline-store.path</name>
<value>${yarn.log.dir}/timeline</value>
</property>
<property>
<description>Handler thread count to serve the client RPC requests.</description>
<name>yarn.timeline-service.handler-thread-count</name>
<value>10</value>
</property>
<property>
<description>Indicate to ResourceManager as well as clients whether
history-service is enabled or not. If enabled, ResourceManager starts
recording historical data that ApplicationHistory service can consume.
Similarly, clients can redirect to the history service when applications
finish if this is enabled.</description>
<name>yarn.timeline-service.generic-application-history.enabled</name>
<value>false</value>
</property>
<property>
<description>URI pointing to the location of the FileSystem path where
the history will be persisted. This must be supplied when using
org.apache.hadoop.yarn.server.applicationhistoryservice.FileSystemApplicationHistoryStore
as the value for yarn.resourcemanager.history-writer.store.class</description>
<name>yarn.ahs.fs-history-store.uri</name>
<value>${hadoop.log.dir}/yarn/system/ahstore</value>
as the value for yarn.timeline-service.generic-application-history.store-class</description>
<name>yarn.timeline-service.generic-application-history.fs-history-store.uri</name>
<value>${hadoop.log.dir}/yarn/system/history</value>
</property>
<property>
<description>This is default address for the Application History server
to start the RPC server.</description>
<name>yarn.ahs.address</name>
<value>0.0.0.0:10200</value>
</property>
<property>
<description>CLient thread count to serve the client requests.</description>
<name>yarn.ahs.client.thread-count</name>
<value>10</value>
</property>
<property>
<description>T-file compression types used to compress history data.</description>
<name>yarn.ahs.fs-history-store.compression-type</name>
<name>yarn.timeline-service.generic-application-history.fs-history-store.compression-type</name>
<value>none</value>
</property>
<property>
<description> Store class name for history store, defaulting to file
system store </description>
<name>yarn.ahs.store.class</name>
<description>Store class name for history store, defaulting to file
system store </description>
<name>yarn.timeline-service.generic-application-history.store-class</name>
<value>org.apache.hadoop.yarn.server.applicationhistoryservice.FileSystemApplicationHistoryStore</value>
</property>
<!-- Application Timeline Service's Configuration-->
<property>
<description>Store class name for application timeline store</description>
<name>yarn.ats.store.class</name>
<value>org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline.LeveldbApplicationTimelineStore</value>
</property>
<property>
<description>Store file name for leveldb application timeline store</description>
<name>yarn.ats.leveldb-apptimeline-store.path</name>
<value>${yarn.log.dir}/ats</value>
</property>
<!-- Other configuration -->
<property>
<description>The interval that the yarn client library uses to poll the

View File

@ -16,7 +16,7 @@
* limitations under the License.
*/
package org.apache.hadoop.yarn.api.records.apptimeline;
package org.apache.hadoop.yarn.api.records.timeline;
import java.util.ArrayList;
import java.util.List;
@ -25,25 +25,30 @@ import junit.framework.Assert;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSPutErrors.ATSPutError;
import org.apache.hadoop.yarn.util.TimelineUtils;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEvents;
import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse.TimelinePutError;
import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
import org.junit.Test;
public class TestApplicationTimelineRecords {
public class TestTimelineRecords {
private static final Log LOG =
LogFactory.getLog(TestApplicationTimelineRecords.class);
LogFactory.getLog(TestTimelineRecords.class);
@Test
public void testATSEntities() throws Exception {
ATSEntities entities = new ATSEntities();
public void testEntities() throws Exception {
TimelineEntities entities = new TimelineEntities();
for (int j = 0; j < 2; ++j) {
ATSEntity entity = new ATSEntity();
TimelineEntity entity = new TimelineEntity();
entity.setEntityId("entity id " + j);
entity.setEntityType("entity type " + j);
entity.setStartTime(System.currentTimeMillis());
for (int i = 0; i < 2; ++i) {
ATSEvent event = new ATSEvent();
TimelineEvent event = new TimelineEvent();
event.setTimestamp(System.currentTimeMillis());
event.setEventType("event type " + i);
event.addEventInfo("key1", "val1");
@ -62,14 +67,14 @@ public class TestApplicationTimelineRecords {
LOG.info(TimelineUtils.dumpTimelineRecordtoJSON(entities, true));
Assert.assertEquals(2, entities.getEntities().size());
ATSEntity entity1 = entities.getEntities().get(0);
TimelineEntity entity1 = entities.getEntities().get(0);
Assert.assertEquals("entity id 0", entity1.getEntityId());
Assert.assertEquals("entity type 0", entity1.getEntityType());
Assert.assertEquals(2, entity1.getRelatedEntities().size());
Assert.assertEquals(2, entity1.getEvents().size());
Assert.assertEquals(2, entity1.getPrimaryFilters().size());
Assert.assertEquals(2, entity1.getOtherInfo().size());
ATSEntity entity2 = entities.getEntities().get(1);
TimelineEntity entity2 = entities.getEntities().get(1);
Assert.assertEquals("entity id 1", entity2.getEntityId());
Assert.assertEquals("entity type 1", entity2.getEntityType());
Assert.assertEquals(2, entity2.getRelatedEntities().size());
@ -79,15 +84,15 @@ public class TestApplicationTimelineRecords {
}
@Test
public void testATSEvents() throws Exception {
ATSEvents events = new ATSEvents();
public void testEvents() throws Exception {
TimelineEvents events = new TimelineEvents();
for (int j = 0; j < 2; ++j) {
ATSEvents.ATSEventsOfOneEntity partEvents =
new ATSEvents.ATSEventsOfOneEntity();
TimelineEvents.EventsOfOneEntity partEvents =
new TimelineEvents.EventsOfOneEntity();
partEvents.setEntityId("entity id " + j);
partEvents.setEntityType("entity type " + j);
for (int i = 0; i < 2; ++i) {
ATSEvent event = new ATSEvent();
TimelineEvent event = new TimelineEvent();
event.setTimestamp(System.currentTimeMillis());
event.setEventType("event type " + i);
event.addEventInfo("key1", "val1");
@ -100,57 +105,57 @@ public class TestApplicationTimelineRecords {
LOG.info(TimelineUtils.dumpTimelineRecordtoJSON(events, true));
Assert.assertEquals(2, events.getAllEvents().size());
ATSEvents.ATSEventsOfOneEntity partEvents1 = events.getAllEvents().get(0);
TimelineEvents.EventsOfOneEntity partEvents1 = events.getAllEvents().get(0);
Assert.assertEquals("entity id 0", partEvents1.getEntityId());
Assert.assertEquals("entity type 0", partEvents1.getEntityType());
Assert.assertEquals(2, partEvents1.getEvents().size());
ATSEvent event11 = partEvents1.getEvents().get(0);
TimelineEvent event11 = partEvents1.getEvents().get(0);
Assert.assertEquals("event type 0", event11.getEventType());
Assert.assertEquals(2, event11.getEventInfo().size());
ATSEvent event12 = partEvents1.getEvents().get(1);
TimelineEvent event12 = partEvents1.getEvents().get(1);
Assert.assertEquals("event type 1", event12.getEventType());
Assert.assertEquals(2, event12.getEventInfo().size());
ATSEvents.ATSEventsOfOneEntity partEvents2 = events.getAllEvents().get(1);
TimelineEvents.EventsOfOneEntity partEvents2 = events.getAllEvents().get(1);
Assert.assertEquals("entity id 1", partEvents2.getEntityId());
Assert.assertEquals("entity type 1", partEvents2.getEntityType());
Assert.assertEquals(2, partEvents2.getEvents().size());
ATSEvent event21 = partEvents2.getEvents().get(0);
TimelineEvent event21 = partEvents2.getEvents().get(0);
Assert.assertEquals("event type 0", event21.getEventType());
Assert.assertEquals(2, event21.getEventInfo().size());
ATSEvent event22 = partEvents2.getEvents().get(1);
TimelineEvent event22 = partEvents2.getEvents().get(1);
Assert.assertEquals("event type 1", event22.getEventType());
Assert.assertEquals(2, event22.getEventInfo().size());
}
@Test
public void testATSPutErrors() throws Exception {
ATSPutErrors atsPutErrors = new ATSPutErrors();
ATSPutError error1 = new ATSPutError();
public void testTimelinePutErrors() throws Exception {
TimelinePutResponse TimelinePutErrors = new TimelinePutResponse();
TimelinePutError error1 = new TimelinePutError();
error1.setEntityId("entity id 1");
error1.setEntityId("entity type 1");
error1.setErrorCode(ATSPutError.NO_START_TIME);
atsPutErrors.addError(error1);
List<ATSPutError> errors = new ArrayList<ATSPutError>();
errors.add(error1);
ATSPutError error2 = new ATSPutError();
error1.setErrorCode(TimelinePutError.NO_START_TIME);
TimelinePutErrors.addError(error1);
List<TimelinePutError> response = new ArrayList<TimelinePutError>();
response.add(error1);
TimelinePutError error2 = new TimelinePutError();
error2.setEntityId("entity id 2");
error2.setEntityId("entity type 2");
error2.setErrorCode(ATSPutError.IO_EXCEPTION);
errors.add(error2);
atsPutErrors.addErrors(errors);
error2.setErrorCode(TimelinePutError.IO_EXCEPTION);
response.add(error2);
TimelinePutErrors.addErrors(response);
LOG.info("Errors in JSON:");
LOG.info(TimelineUtils.dumpTimelineRecordtoJSON(atsPutErrors, true));
LOG.info(TimelineUtils.dumpTimelineRecordtoJSON(TimelinePutErrors, true));
Assert.assertEquals(3, atsPutErrors.getErrors().size());
ATSPutError e = atsPutErrors.getErrors().get(0);
Assert.assertEquals(3, TimelinePutErrors.getErrors().size());
TimelinePutError e = TimelinePutErrors.getErrors().get(0);
Assert.assertEquals(error1.getEntityId(), e.getEntityId());
Assert.assertEquals(error1.getEntityType(), e.getEntityType());
Assert.assertEquals(error1.getErrorCode(), e.getErrorCode());
e = atsPutErrors.getErrors().get(1);
e = TimelinePutErrors.getErrors().get(1);
Assert.assertEquals(error1.getEntityId(), e.getEntityId());
Assert.assertEquals(error1.getEntityType(), e.getEntityType());
Assert.assertEquals(error1.getErrorCode(), e.getErrorCode());
e = atsPutErrors.getErrors().get(2);
e = TimelinePutErrors.getErrors().get(2);
Assert.assertEquals(error2.getEntityId(), e.getEntityId());
Assert.assertEquals(error2.getEntityType(), e.getEntityType());
Assert.assertEquals(error2.getErrorCode(), e.getErrorCode());

View File

@ -76,19 +76,19 @@ public class ApplicationHistoryClientService extends AbstractService {
Configuration conf = getConfig();
YarnRPC rpc = YarnRPC.create(conf);
InetSocketAddress address =
conf.getSocketAddr(YarnConfiguration.AHS_ADDRESS,
YarnConfiguration.DEFAULT_AHS_ADDRESS,
YarnConfiguration.DEFAULT_AHS_PORT);
conf.getSocketAddr(YarnConfiguration.TIMELINE_SERVICE_ADDRESS,
YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ADDRESS,
YarnConfiguration.DEFAULT_TIMELINE_SERVICE_PORT);
server =
rpc.getServer(ApplicationHistoryProtocol.class, protocolHandler,
address, conf, null, conf.getInt(
YarnConfiguration.AHS_CLIENT_THREAD_COUNT,
YarnConfiguration.DEFAULT_AHS_CLIENT_THREAD_COUNT));
YarnConfiguration.TIMELINE_SERVICE_HANDLER_THREAD_COUNT,
YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_THREAD_COUNT));
server.start();
this.bindAddress =
conf.updateConnectAddr(YarnConfiguration.AHS_ADDRESS,
conf.updateConnectAddr(YarnConfiguration.TIMELINE_SERVICE_ADDRESS,
server.getListenerAddress());
LOG.info("Instantiated ApplicationHistoryClientService at "
+ this.bindAddress);

View File

@ -79,7 +79,8 @@ public class ApplicationHistoryManagerImpl extends AbstractService implements
protected ApplicationHistoryStore createApplicationHistoryStore(
Configuration conf) {
return ReflectionUtils.newInstance(conf.getClass(
YarnConfiguration.AHS_STORE, FileSystemApplicationHistoryStore.class,
YarnConfiguration.APPLICATION_HISTORY_STORE,
FileSystemApplicationHistoryStore.class,
ApplicationHistoryStore.class), conf);
}

View File

@ -33,8 +33,8 @@ import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline.ApplicationTimelineStore;
import org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline.LeveldbApplicationTimelineStore;
import org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.TimelineStore;
import org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.LeveldbTimelineStore;
import org.apache.hadoop.yarn.server.applicationhistoryservice.webapp.AHSWebApp;
import org.apache.hadoop.yarn.webapp.WebApp;
import org.apache.hadoop.yarn.webapp.WebApps;
@ -54,7 +54,7 @@ public class ApplicationHistoryServer extends CompositeService {
ApplicationHistoryClientService ahsClientService;
ApplicationHistoryManager historyManager;
ApplicationTimelineStore timelineStore;
TimelineStore timelineStore;
private WebApp webApp;
public ApplicationHistoryServer() {
@ -67,7 +67,7 @@ public class ApplicationHistoryServer extends CompositeService {
ahsClientService = createApplicationHistoryClientService(historyManager);
addService(ahsClientService);
addService((Service) historyManager);
timelineStore = createApplicationTimelineStore(conf);
timelineStore = createTimelineStore(conf);
addIfService(timelineStore);
super.serviceInit(conf);
}
@ -141,11 +141,11 @@ public class ApplicationHistoryServer extends CompositeService {
return new ApplicationHistoryManagerImpl();
}
protected ApplicationTimelineStore createApplicationTimelineStore(
protected TimelineStore createTimelineStore(
Configuration conf) {
return ReflectionUtils.newInstance(conf.getClass(
YarnConfiguration.ATS_STORE, LeveldbApplicationTimelineStore.class,
ApplicationTimelineStore.class), conf);
YarnConfiguration.TIMELINE_SERVICE_STORE, LeveldbTimelineStore.class,
TimelineStore.class), conf);
}
protected void startWebApp() {
@ -158,9 +158,9 @@ public class ApplicationHistoryServer extends CompositeService {
ahsClientService, "ws")
.with(getConfig())
.withHttpSpnegoPrincipalKey(
YarnConfiguration.AHS_WEBAPP_SPNEGO_USER_NAME_KEY)
YarnConfiguration.TIMELINE_SERVICE_WEBAPP_SPNEGO_USER_NAME_KEY)
.withHttpSpnegoKeytabKey(
YarnConfiguration.AHS_WEBAPP_SPNEGO_KEYTAB_FILE_KEY)
YarnConfiguration.TIMELINE_SERVICE_WEBAPP_SPNEGO_KEYTAB_FILE_KEY)
.at(bindAddress)
.start(new AHSWebApp(historyManager, timelineStore));
} catch (Exception e) {

View File

@ -113,7 +113,7 @@ public class FileSystemApplicationHistoryStore extends AbstractService
@Override
public void serviceInit(Configuration conf) throws Exception {
Path fsWorkingPath =
new Path(conf.get(YarnConfiguration.FS_HISTORY_STORE_URI));
new Path(conf.get(YarnConfiguration.FS_APPLICATION_HISTORY_STORE_URI));
rootDirPath = new Path(fsWorkingPath, ROOT_DIR_NAME);
try {
fs = fsWorkingPath.getFileSystem(conf);
@ -727,8 +727,8 @@ public class FileSystemApplicationHistoryStore extends AbstractService
fs.setPermission(historyFile, HISTORY_FILE_UMASK);
writer =
new TFile.Writer(fsdos, MIN_BLOCK_SIZE, getConfig().get(
YarnConfiguration.FS_HISTORY_STORE_COMPRESSION_TYPE,
YarnConfiguration.DEFAULT_FS_HISTORY_STORE_COMPRESSION_TYPE), null,
YarnConfiguration.FS_APPLICATION_HISTORY_STORE_COMPRESSION_TYPE,
YarnConfiguration.DEFAULT_FS_APPLICATION_HISTORY_STORE_COMPRESSION_TYPE), null,
getConfig());
}

View File

@ -1,131 +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.hadoop.yarn.server.applicationhistoryservice.apptimeline;
import java.io.IOException;
import java.util.Collection;
import java.util.EnumSet;
import java.util.Set;
import java.util.SortedSet;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSEntities;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSEntity;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSEvents;
/**
* This interface is for retrieving application timeline information.
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
public interface ApplicationTimelineReader {
/**
* Possible fields to retrieve for {@link #getEntities} and {@link
* #getEntity}.
*/
enum Field {
EVENTS,
RELATED_ENTITIES,
PRIMARY_FILTERS,
OTHER_INFO,
LAST_EVENT_ONLY
}
/**
* Default limit for {@link #getEntities} and {@link #getEntityTimelines}.
*/
final long DEFAULT_LIMIT = 100;
/**
* This method retrieves a list of entity information, {@link ATSEntity},
* sorted by the starting timestamp for the entity, descending.
*
* @param entityType The type of entities to return (required).
* @param limit A limit on the number of entities to return. If null,
* defaults to {@link #DEFAULT_LIMIT}.
* @param windowStart The earliest start timestamp to retrieve (exclusive).
* If null, defaults to retrieving all entities until the
* limit is reached.
* @param windowEnd The latest start timestamp to retrieve (inclusive).
* If null, defaults to {@link Long#MAX_VALUE}
* @param primaryFilter Retrieves only entities that have the specified
* primary filter. If null, retrieves all entities.
* This is an indexed retrieval, and no entities that
* do not match the filter are scanned.
* @param secondaryFilters Retrieves only entities that have exact matches
* for all the specified filters in their primary
* filters or other info. This is not an indexed
* retrieval, so all entities are scanned but only
* those matching the filters are returned.
* @param fieldsToRetrieve Specifies which fields of the entity object to
* retrieve (see {@link Field}). If the set of fields
* contains {@link Field#LAST_EVENT_ONLY} and not
* {@link Field#EVENTS}, the most recent event for
* each entity is retrieved. If null, retrieves all
* fields.
* @return An {@link ATSEntities} object.
* @throws IOException
*/
ATSEntities getEntities(String entityType,
Long limit, Long windowStart, Long windowEnd,
NameValuePair primaryFilter, Collection<NameValuePair> secondaryFilters,
EnumSet<Field> fieldsToRetrieve) throws IOException;
/**
* This method retrieves the entity information for a given entity.
*
* @param entity The entity whose information will be retrieved.
* @param entityType The type of the entity.
* @param fieldsToRetrieve Specifies which fields of the entity object to
* retrieve (see {@link Field}). If the set of
* fields contains {@link Field#LAST_EVENT_ONLY} and
* not {@link Field#EVENTS}, the most recent event
* for each entity is retrieved. If null, retrieves
* all fields.
* @return An {@link ATSEntity} object.
* @throws IOException
*/
ATSEntity getEntity(String entity, String entityType, EnumSet<Field>
fieldsToRetrieve) throws IOException;
/**
* This method retrieves the events for a list of entities all of the same
* entity type. The events for each entity are sorted in order of their
* timestamps, descending.
*
* @param entityType The type of entities to retrieve events for.
* @param entityIds The entity IDs to retrieve events for.
* @param limit A limit on the number of events to return for each entity.
* If null, defaults to {@link #DEFAULT_LIMIT} events per
* entity.
* @param windowStart If not null, retrieves only events later than the
* given time (exclusive)
* @param windowEnd If not null, retrieves only events earlier than the
* given time (inclusive)
* @param eventTypes Restricts the events returned to the given types. If
* null, events of all types will be returned.
* @return An {@link ATSEvents} object.
* @throws IOException
*/
ATSEvents getEntityTimelines(String entityType,
SortedSet<String> entityIds, Long limit, Long windowStart,
Long windowEnd, Set<String> eventTypes) throws IOException;
}

View File

@ -16,7 +16,7 @@
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline;
package org.apache.hadoop.yarn.server.applicationhistoryservice.timeline;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceStability.Unstable;

View File

@ -15,12 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.io.WritableUtils;
import org.codehaus.jackson.map.ObjectMapper;
package org.apache.hadoop.yarn.server.applicationhistoryservice.timeline;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
@ -31,14 +26,18 @@ import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.io.WritableUtils;
import org.codehaus.jackson.map.ObjectMapper;
/**
* A utility class providing methods for serializing and deserializing
* objects. The {@link #write(Object)}, {@link #read(byte[])} and {@link
* #write(java.io.DataOutputStream, Object)}, {@link
* #read(java.io.DataInputStream)} methods are used by the
* {@link LeveldbApplicationTimelineStore} to store and retrieve arbitrary
* {@link LeveldbTimelineStore} to store and retrieve arbitrary
* JSON, while the {@link #writeReverseOrderedLong} and {@link
* #readReverseOrderedLong} methods are used to sort entities in descending
* start time order.
@ -132,9 +131,24 @@ public class GenericObjectMapper {
* @throws IOException
*/
public static Object read(byte[] b) throws IOException {
if (b == null || b.length == 0)
return read(b, 0);
}
/**
* Deserializes an Object from a byte array at a specified offset, assuming
* the bytes were created with {@link #write(Object)}.
*
* @param b A byte array
* @param offset Offset into the array
* @return An Object
* @throws IOException
*/
public static Object read(byte[] b, int offset) throws IOException {
if (b == null || b.length == 0) {
return null;
ByteArrayInputStream bais = new ByteArrayInputStream(b);
}
ByteArrayInputStream bais = new ByteArrayInputStream(b, offset,
b.length - offset);
return read(new DataInputStream(bais));
}

View File

@ -16,7 +16,7 @@
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline;
package org.apache.hadoop.yarn.server.applicationhistoryservice.timeline;
import java.io.ByteArrayOutputStream;
import java.io.File;
@ -44,13 +44,13 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.WritableComparator;
import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSEntities;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSEntity;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSEvent;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSEvents;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSEvents.ATSEventsOfOneEntity;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSPutErrors;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSPutErrors.ATSPutError;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEvents;
import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEvents.EventsOfOneEntity;
import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse.TimelinePutError;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.fusesource.leveldbjni.JniDBFactory;
import org.iq80.leveldb.DB;
@ -58,22 +58,20 @@ import org.iq80.leveldb.DBIterator;
import org.iq80.leveldb.Options;
import org.iq80.leveldb.WriteBatch;
import static org.apache.hadoop.yarn.server.applicationhistoryservice
.apptimeline.GenericObjectMapper.readReverseOrderedLong;
import static org.apache.hadoop.yarn.server.applicationhistoryservice
.apptimeline.GenericObjectMapper.writeReverseOrderedLong;
import static org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.GenericObjectMapper.readReverseOrderedLong;
import static org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.GenericObjectMapper.writeReverseOrderedLong;
/**
* An implementation of an application timeline store backed by leveldb.
* An implementation of a timeline store backed by leveldb.
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
public class LeveldbApplicationTimelineStore extends AbstractService
implements ApplicationTimelineStore {
public class LeveldbTimelineStore extends AbstractService
implements TimelineStore {
private static final Log LOG = LogFactory
.getLog(LeveldbApplicationTimelineStore.class);
.getLog(LeveldbTimelineStore.class);
private static final String FILENAME = "leveldb-apptimeline-store.ldb";
private static final String FILENAME = "leveldb-timeline-store.ldb";
private static final byte[] START_TIME_LOOKUP_PREFIX = "k".getBytes();
private static final byte[] ENTITY_ENTRY_PREFIX = "e".getBytes();
@ -94,8 +92,8 @@ public class LeveldbApplicationTimelineStore extends AbstractService
private DB db;
public LeveldbApplicationTimelineStore() {
super(LeveldbApplicationTimelineStore.class.getName());
public LeveldbTimelineStore() {
super(LeveldbTimelineStore.class.getName());
}
@Override
@ -103,12 +101,12 @@ public class LeveldbApplicationTimelineStore extends AbstractService
Options options = new Options();
options.createIfMissing(true);
JniDBFactory factory = new JniDBFactory();
String path = conf.get(YarnConfiguration.ATS_LEVELDB_PATH_PROPERTY);
String path = conf.get(YarnConfiguration.TIMELINE_SERVICE_LEVELDB_PATH);
File p = new File(path);
if (!p.exists())
if (!p.mkdirs())
throw new IOException("Couldn't create directory for leveldb " +
"application timeline store " + path);
"timeline store " + path);
LOG.info("Using leveldb path " + path);
db = factory.open(new File(path, FILENAME), options);
super.serviceInit(conf);
@ -212,20 +210,20 @@ public class LeveldbApplicationTimelineStore extends AbstractService
}
@Override
public ATSEntity getEntity(String entity, String entityType,
public TimelineEntity getEntity(String entityId, String entityType,
EnumSet<Field> fields) throws IOException {
DBIterator iterator = null;
try {
byte[] revStartTime = getStartTime(entity, entityType, null, null, null);
byte[] revStartTime = getStartTime(entityId, entityType, null, null, null);
if (revStartTime == null)
return null;
byte[] prefix = KeyBuilder.newInstance().add(ENTITY_ENTRY_PREFIX)
.add(entityType).add(revStartTime).add(entity).getBytesForLookup();
.add(entityType).add(revStartTime).add(entityId).getBytesForLookup();
iterator = db.iterator();
iterator.seek(prefix);
return getEntity(entity, entityType,
return getEntity(entityId, entityType,
readReverseOrderedLong(revStartTime, 0), fields, iterator, prefix,
prefix.length);
} finally {
@ -237,45 +235,43 @@ public class LeveldbApplicationTimelineStore extends AbstractService
* Read entity from a db iterator. If no information is found in the
* specified fields for this entity, return null.
*/
private static ATSEntity getEntity(String entity, String entityType,
private static TimelineEntity getEntity(String entityId, String entityType,
Long startTime, EnumSet<Field> fields, DBIterator iterator,
byte[] prefix, int prefixlen) throws IOException {
if (fields == null)
fields = EnumSet.allOf(Field.class);
ATSEntity atsEntity = new ATSEntity();
TimelineEntity entity = new TimelineEntity();
boolean events = false;
boolean lastEvent = false;
if (fields.contains(Field.EVENTS)) {
events = true;
atsEntity.setEvents(new ArrayList<ATSEvent>());
entity.setEvents(new ArrayList<TimelineEvent>());
} else if (fields.contains(Field.LAST_EVENT_ONLY)) {
lastEvent = true;
atsEntity.setEvents(new ArrayList<ATSEvent>());
entity.setEvents(new ArrayList<TimelineEvent>());
}
else {
atsEntity.setEvents(null);
entity.setEvents(null);
}
boolean relatedEntities = false;
if (fields.contains(Field.RELATED_ENTITIES)) {
relatedEntities = true;
atsEntity.setRelatedEntities(new HashMap<String, List<String>>());
} else {
atsEntity.setRelatedEntities(null);
entity.setRelatedEntities(null);
}
boolean primaryFilters = false;
if (fields.contains(Field.PRIMARY_FILTERS)) {
primaryFilters = true;
atsEntity.setPrimaryFilters(new HashMap<String, Object>());
} else {
atsEntity.setPrimaryFilters(null);
entity.setPrimaryFilters(null);
}
boolean otherInfo = false;
if (fields.contains(Field.OTHER_INFO)) {
otherInfo = true;
atsEntity.setOtherInfo(new HashMap<String, Object>());
entity.setOtherInfo(new HashMap<String, Object>());
} else {
atsEntity.setOtherInfo(null);
entity.setOtherInfo(null);
}
// iterate through the entity's entry, parsing information if it is part
@ -286,49 +282,48 @@ public class LeveldbApplicationTimelineStore extends AbstractService
break;
if (key[prefixlen] == PRIMARY_FILTER_COLUMN[0]) {
if (primaryFilters) {
atsEntity.addPrimaryFilter(parseRemainingKey(key,
prefixlen + PRIMARY_FILTER_COLUMN.length),
GenericObjectMapper.read(iterator.peekNext().getValue()));
addPrimaryFilter(entity, key,
prefixlen + PRIMARY_FILTER_COLUMN.length);
}
} else if (key[prefixlen] == OTHER_INFO_COLUMN[0]) {
if (otherInfo) {
atsEntity.addOtherInfo(parseRemainingKey(key,
entity.addOtherInfo(parseRemainingKey(key,
prefixlen + OTHER_INFO_COLUMN.length),
GenericObjectMapper.read(iterator.peekNext().getValue()));
}
} else if (key[prefixlen] == RELATED_COLUMN[0]) {
if (relatedEntities) {
addRelatedEntity(atsEntity, key,
addRelatedEntity(entity, key,
prefixlen + RELATED_COLUMN.length);
}
} else if (key[prefixlen] == TIME_COLUMN[0]) {
if (events || (lastEvent && atsEntity.getEvents().size() == 0)) {
ATSEvent event = getEntityEvent(null, key, prefixlen +
if (events || (lastEvent && entity.getEvents().size() == 0)) {
TimelineEvent event = getEntityEvent(null, key, prefixlen +
TIME_COLUMN.length, iterator.peekNext().getValue());
if (event != null) {
atsEntity.addEvent(event);
entity.addEvent(event);
}
}
} else {
LOG.warn(String.format("Found unexpected column for entity %s of " +
"type %s (0x%02x)", entity, entityType, key[prefixlen]));
"type %s (0x%02x)", entityId, entityType, key[prefixlen]));
}
}
atsEntity.setEntityId(entity);
atsEntity.setEntityType(entityType);
atsEntity.setStartTime(startTime);
entity.setEntityId(entityId);
entity.setEntityType(entityType);
entity.setStartTime(startTime);
return atsEntity;
return entity;
}
@Override
public ATSEvents getEntityTimelines(String entityType,
public TimelineEvents getEntityTimelines(String entityType,
SortedSet<String> entityIds, Long limit, Long windowStart,
Long windowEnd, Set<String> eventType) throws IOException {
ATSEvents atsEvents = new ATSEvents();
TimelineEvents events = new TimelineEvents();
if (entityIds == null || entityIds.isEmpty())
return atsEvents;
return events;
// create a lexicographically-ordered map from start time to entities
Map<byte[], List<EntityIdentifier>> startTimeMap = new TreeMap<byte[],
List<EntityIdentifier>>(new Comparator<byte[]>() {
@ -359,13 +354,13 @@ public class LeveldbApplicationTimelineStore extends AbstractService
// start time, end time, event types) for entities whose start times
// were found and add the entities to the return list
byte[] revStartTime = entry.getKey();
for (EntityIdentifier entity : entry.getValue()) {
ATSEventsOfOneEntity atsEntity = new ATSEventsOfOneEntity();
atsEntity.setEntityId(entity.getId());
atsEntity.setEntityType(entityType);
atsEvents.addEvent(atsEntity);
for (EntityIdentifier entityID : entry.getValue()) {
EventsOfOneEntity entity = new EventsOfOneEntity();
entity.setEntityId(entityID.getId());
entity.setEntityType(entityType);
events.addEvent(entity);
KeyBuilder kb = KeyBuilder.newInstance().add(ENTITY_ENTRY_PREFIX)
.add(entityType).add(revStartTime).add(entity.getId())
.add(entityType).add(revStartTime).add(entityID.getId())
.add(TIME_COLUMN);
byte[] prefix = kb.getBytesForLookup();
if (windowEnd == null) {
@ -383,24 +378,24 @@ public class LeveldbApplicationTimelineStore extends AbstractService
limit = DEFAULT_LIMIT;
}
iterator = db.iterator();
for (iterator.seek(first); atsEntity.getEvents().size() < limit &&
for (iterator.seek(first); entity.getEvents().size() < limit &&
iterator.hasNext(); iterator.next()) {
byte[] key = iterator.peekNext().getKey();
if (!prefixMatches(prefix, prefix.length, key) || (last != null &&
WritableComparator.compareBytes(key, 0, key.length, last, 0,
last.length) > 0))
break;
ATSEvent event = getEntityEvent(eventType, key, prefix.length,
TimelineEvent event = getEntityEvent(eventType, key, prefix.length,
iterator.peekNext().getValue());
if (event != null)
atsEntity.addEvent(event);
entity.addEvent(event);
}
}
}
} finally {
IOUtils.cleanup(LOG, iterator);
}
return atsEvents;
return events;
}
/**
@ -415,7 +410,7 @@ public class LeveldbApplicationTimelineStore extends AbstractService
}
@Override
public ATSEntities getEntities(String entityType,
public TimelineEntities getEntities(String entityType,
Long limit, Long windowStart, Long windowEnd,
NameValuePair primaryFilter, Collection<NameValuePair> secondaryFilters,
EnumSet<Field> fields) throws IOException {
@ -450,7 +445,7 @@ public class LeveldbApplicationTimelineStore extends AbstractService
* @return A list of entities
* @throws IOException
*/
private ATSEntities getEntityByTime(byte[] base,
private TimelineEntities getEntityByTime(byte[] base,
String entityType, Long limit, Long starttime, Long endtime,
Collection<NameValuePair> secondaryFilters, EnumSet<Field> fields)
throws IOException {
@ -479,82 +474,87 @@ public class LeveldbApplicationTimelineStore extends AbstractService
limit = DEFAULT_LIMIT;
}
ATSEntities atsEntities = new ATSEntities();
TimelineEntities entities = new TimelineEntities();
iterator = db.iterator();
iterator.seek(first);
// iterate until one of the following conditions is met: limit is
// reached, there are no more keys, the key prefix no longer matches,
// or a start time has been specified and reached/exceeded
while (atsEntities.getEntities().size() < limit && iterator.hasNext()) {
while (entities.getEntities().size() < limit && iterator.hasNext()) {
byte[] key = iterator.peekNext().getKey();
if (!prefixMatches(prefix, prefix.length, key) || (last != null &&
WritableComparator.compareBytes(key, 0, key.length, last, 0,
last.length) > 0))
break;
// read the start time and entity from the current key
// read the start time and entityId from the current key
KeyParser kp = new KeyParser(key, prefix.length);
Long startTime = kp.getNextLong();
String entity = kp.getNextString();
String entityId = kp.getNextString();
// parse the entity that owns this key, iterating over all keys for
// the entity
ATSEntity atsEntity = getEntity(entity, entityType, startTime,
TimelineEntity entity = getEntity(entityId, entityType, startTime,
fields, iterator, key, kp.getOffset());
if (atsEntity == null)
if (entity == null)
continue;
// determine if the retrieved entity matches the provided secondary
// filters, and if so add it to the list of entities to return
boolean filterPassed = true;
if (secondaryFilters != null) {
for (NameValuePair filter : secondaryFilters) {
Object v = atsEntity.getOtherInfo().get(filter.getName());
if (v == null)
v = atsEntity.getPrimaryFilters().get(filter.getName());
if (v == null || !v.equals(filter.getValue())) {
Object v = entity.getOtherInfo().get(filter.getName());
if (v == null) {
Set<Object> vs = entity.getPrimaryFilters()
.get(filter.getName());
if (vs != null && !vs.contains(filter.getValue())) {
filterPassed = false;
break;
}
} else if (!v.equals(filter.getValue())) {
filterPassed = false;
break;
}
}
}
if (filterPassed)
atsEntities.addEntity(atsEntity);
entities.addEntity(entity);
}
return atsEntities;
return entities;
} finally {
IOUtils.cleanup(LOG, iterator);
}
}
/**
* Put a single entity. If there is an error, add a PutError to the given
* Put a single entity. If there is an error, add a TimelinePutError to the given
* response.
*/
private void put(ATSEntity atsEntity, ATSPutErrors response) {
private void put(TimelineEntity entity, TimelinePutResponse response) {
WriteBatch writeBatch = null;
try {
writeBatch = db.createWriteBatch();
List<ATSEvent> events = atsEntity.getEvents();
List<TimelineEvent> events = entity.getEvents();
// look up the start time for the entity
byte[] revStartTime = getStartTime(atsEntity.getEntityId(),
atsEntity.getEntityType(), atsEntity.getStartTime(), events,
byte[] revStartTime = getStartTime(entity.getEntityId(),
entity.getEntityType(), entity.getStartTime(), events,
writeBatch);
if (revStartTime == null) {
// if no start time is found, add an error and return
ATSPutError error = new ATSPutError();
error.setEntityId(atsEntity.getEntityId());
error.setEntityType(atsEntity.getEntityType());
error.setErrorCode(ATSPutError.NO_START_TIME);
TimelinePutError error = new TimelinePutError();
error.setEntityId(entity.getEntityId());
error.setEntityType(entity.getEntityType());
error.setErrorCode(TimelinePutError.NO_START_TIME);
response.addError(error);
return;
}
Long revStartTimeLong = readReverseOrderedLong(revStartTime, 0);
Map<String, Object> primaryFilters = atsEntity.getPrimaryFilters();
Map<String, Set<Object>> primaryFilters = entity.getPrimaryFilters();
// write event entries
if (events != null && !events.isEmpty()) {
for (ATSEvent event : events) {
for (TimelineEvent event : events) {
byte[] revts = writeReverseOrderedLong(event.getTimestamp());
byte[] key = createEntityEventKey(atsEntity.getEntityId(),
atsEntity.getEntityType(), revStartTime, revts,
byte[] key = createEntityEventKey(entity.getEntityId(),
entity.getEntityType(), revStartTime, revts,
event.getEventType());
byte[] value = GenericObjectMapper.write(event.getEventInfo());
writeBatch.put(key, value);
@ -563,10 +563,10 @@ public class LeveldbApplicationTimelineStore extends AbstractService
}
// write related entity entries
Map<String,List<String>> relatedEntities =
atsEntity.getRelatedEntities();
Map<String, Set<String>> relatedEntities =
entity.getRelatedEntities();
if (relatedEntities != null && !relatedEntities.isEmpty()) {
for (Entry<String, List<String>> relatedEntityList :
for (Entry<String, Set<String>> relatedEntityList :
relatedEntities.entrySet()) {
String relatedEntityType = relatedEntityList.getKey();
for (String relatedEntityId : relatedEntityList.getValue()) {
@ -586,7 +586,7 @@ public class LeveldbApplicationTimelineStore extends AbstractService
// write reverse entry (related entity -> entity)
byte[] key = createReleatedEntityKey(relatedEntityId,
relatedEntityType, relatedEntityStartTime,
atsEntity.getEntityId(), atsEntity.getEntityType());
entity.getEntityId(), entity.getEntityType());
writeBatch.put(key, EMPTY_BYTES);
// TODO: write forward entry (entity -> related entity)?
}
@ -595,21 +595,25 @@ public class LeveldbApplicationTimelineStore extends AbstractService
// write primary filter entries
if (primaryFilters != null && !primaryFilters.isEmpty()) {
for (Entry<String, Object> primaryFilter : primaryFilters.entrySet()) {
byte[] key = createPrimaryFilterKey(atsEntity.getEntityId(),
atsEntity.getEntityType(), revStartTime, primaryFilter.getKey());
byte[] value = GenericObjectMapper.write(primaryFilter.getValue());
writeBatch.put(key, value);
writePrimaryFilterEntries(writeBatch, primaryFilters, key, value);
for (Entry<String, Set<Object>> primaryFilter :
primaryFilters.entrySet()) {
for (Object primaryFilterValue : primaryFilter.getValue()) {
byte[] key = createPrimaryFilterKey(entity.getEntityId(),
entity.getEntityType(), revStartTime,
primaryFilter.getKey(), primaryFilterValue);
writeBatch.put(key, EMPTY_BYTES);
writePrimaryFilterEntries(writeBatch, primaryFilters, key,
EMPTY_BYTES);
}
}
}
// write other info entries
Map<String, Object> otherInfo = atsEntity.getOtherInfo();
Map<String, Object> otherInfo = entity.getOtherInfo();
if (otherInfo != null && !otherInfo.isEmpty()) {
for (Entry<String, Object> i : otherInfo.entrySet()) {
byte[] key = createOtherInfoKey(atsEntity.getEntityId(),
atsEntity.getEntityType(), revStartTime, i.getKey());
byte[] key = createOtherInfoKey(entity.getEntityId(),
entity.getEntityType(), revStartTime, i.getKey());
byte[] value = GenericObjectMapper.write(i.getValue());
writeBatch.put(key, value);
writePrimaryFilterEntries(writeBatch, primaryFilters, key, value);
@ -617,12 +621,12 @@ public class LeveldbApplicationTimelineStore extends AbstractService
}
db.write(writeBatch);
} catch (IOException e) {
LOG.error("Error putting entity " + atsEntity.getEntityId() +
" of type " + atsEntity.getEntityType(), e);
ATSPutError error = new ATSPutError();
error.setEntityId(atsEntity.getEntityId());
error.setEntityType(atsEntity.getEntityType());
error.setErrorCode(ATSPutError.IO_EXCEPTION);
LOG.error("Error putting entity " + entity.getEntityId() +
" of type " + entity.getEntityType(), e);
TimelinePutError error = new TimelinePutError();
error.setEntityId(entity.getEntityId());
error.setEntityType(entity.getEntityType());
error.setErrorCode(TimelinePutError.IO_EXCEPTION);
response.addError(error);
} finally {
IOUtils.cleanup(LOG, writeBatch);
@ -634,21 +638,23 @@ public class LeveldbApplicationTimelineStore extends AbstractService
* write additional entries to the db for each primary filter.
*/
private static void writePrimaryFilterEntries(WriteBatch writeBatch,
Map<String, Object> primaryFilters, byte[] key, byte[] value)
Map<String, Set<Object>> primaryFilters, byte[] key, byte[] value)
throws IOException {
if (primaryFilters != null && !primaryFilters.isEmpty()) {
for (Entry<String, Object> p : primaryFilters.entrySet()) {
writeBatch.put(addPrimaryFilterToKey(p.getKey(), p.getValue(),
key), value);
for (Entry<String, Set<Object>> pf : primaryFilters.entrySet()) {
for (Object pfval : pf.getValue()) {
writeBatch.put(addPrimaryFilterToKey(pf.getKey(), pfval,
key), value);
}
}
}
}
@Override
public ATSPutErrors put(ATSEntities atsEntities) {
ATSPutErrors response = new ATSPutErrors();
for (ATSEntity atsEntity : atsEntities.getEntities()) {
put(atsEntity, response);
public TimelinePutResponse put(TimelineEntities entities) {
TimelinePutResponse response = new TimelinePutResponse();
for (TimelineEntity entity : entities.getEntities()) {
put(entity, response);
}
return response;
}
@ -668,7 +674,7 @@ public class LeveldbApplicationTimelineStore extends AbstractService
* @throws IOException
*/
private byte[] getStartTime(String entityId, String entityType,
Long startTime, List<ATSEvent> events, WriteBatch writeBatch)
Long startTime, List<TimelineEvent> events, WriteBatch writeBatch)
throws IOException {
EntityIdentifier entity = new EntityIdentifier(entityId, entityType);
if (startTime == null) {
@ -688,7 +694,7 @@ public class LeveldbApplicationTimelineStore extends AbstractService
return null;
}
Long min = Long.MAX_VALUE;
for (ATSEvent e : events)
for (TimelineEvent e : events)
if (min > e.getTimestamp())
min = e.getTimestamp();
startTime = min;
@ -764,13 +770,13 @@ public class LeveldbApplicationTimelineStore extends AbstractService
* event type is not contained in the specified set of event types,
* returns null.
*/
private static ATSEvent getEntityEvent(Set<String> eventTypes, byte[] key,
private static TimelineEvent getEntityEvent(Set<String> eventTypes, byte[] key,
int offset, byte[] value) throws IOException {
KeyParser kp = new KeyParser(key, offset);
long ts = kp.getNextLong();
String tstype = kp.getNextString();
if (eventTypes == null || eventTypes.contains(tstype)) {
ATSEvent event = new ATSEvent();
TimelineEvent event = new TimelineEvent();
event.setTimestamp(ts);
event.setEventType(tstype);
Object o = GenericObjectMapper.read(value);
@ -790,13 +796,26 @@ public class LeveldbApplicationTimelineStore extends AbstractService
/**
* Creates a primary filter key, serializing ENTITY_ENTRY_PREFIX +
* entitytype + revstarttime + entity + PRIMARY_FILTER_COLUMN + name.
* entitytype + revstarttime + entity + PRIMARY_FILTER_COLUMN + name + value.
*/
private static byte[] createPrimaryFilterKey(String entity,
String entitytype, byte[] revStartTime, String name) throws IOException {
String entitytype, byte[] revStartTime, String name, Object value)
throws IOException {
return KeyBuilder.newInstance().add(ENTITY_ENTRY_PREFIX).add(entitytype)
.add(revStartTime).add(entity).add(PRIMARY_FILTER_COLUMN).add(name)
.getBytes();
.add(GenericObjectMapper.write(value)).getBytes();
}
/**
* Parses the primary filter from the given key at the given offset and
* adds it to the given entity.
*/
private static void addPrimaryFilter(TimelineEntity entity, byte[] key,
int offset) throws IOException {
KeyParser kp = new KeyParser(key, offset);
String name = kp.getNextString();
Object value = GenericObjectMapper.read(key, kp.getOffset());
entity.addPrimaryFilter(name, value);
}
/**
@ -835,12 +854,12 @@ public class LeveldbApplicationTimelineStore extends AbstractService
* Parses the related entity from the given key at the given offset and
* adds it to the given entity.
*/
private static void addRelatedEntity(ATSEntity atsEntity, byte[] key,
private static void addRelatedEntity(TimelineEntity entity, byte[] key,
int offset) throws IOException {
KeyParser kp = new KeyParser(key, offset);
String type = kp.getNextString();
String id = kp.getNextString();
atsEntity.addRelatedEntity(type, id);
entity.addRelatedEntity(type, id);
}
/**

View File

@ -16,7 +16,7 @@
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline;
package org.apache.hadoop.yarn.server.applicationhistoryservice.timeline;
import java.util.ArrayList;
import java.util.Arrays;
@ -33,16 +33,16 @@ import java.util.SortedSet;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSEntities;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSEntity;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSEvent;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSEvents;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSEvents.ATSEventsOfOneEntity;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSPutErrors;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSPutErrors.ATSPutError;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEvents;
import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEvents.EventsOfOneEntity;
import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse.TimelinePutError;
/**
* In-memory implementation of {@link ApplicationTimelineStore}. This
* In-memory implementation of {@link TimelineStore}. This
* implementation is for test purpose only. If users improperly instantiate it,
* they may encounter reading and writing history data in different memory
* store.
@ -50,18 +50,18 @@ import org.apache.hadoop.yarn.api.records.apptimeline.ATSPutErrors.ATSPutError;
*/
@Private
@Unstable
public class MemoryApplicationTimelineStore
extends AbstractService implements ApplicationTimelineStore {
public class MemoryTimelineStore
extends AbstractService implements TimelineStore {
private Map<EntityIdentifier, ATSEntity> entities =
new HashMap<EntityIdentifier, ATSEntity>();
private Map<EntityIdentifier, TimelineEntity> entities =
new HashMap<EntityIdentifier, TimelineEntity>();
public MemoryApplicationTimelineStore() {
super(MemoryApplicationTimelineStore.class.getName());
public MemoryTimelineStore() {
super(MemoryTimelineStore.class.getName());
}
@Override
public ATSEntities getEntities(String entityType, Long limit,
public TimelineEntities getEntities(String entityType, Long limit,
Long windowStart, Long windowEnd, NameValuePair primaryFilter,
Collection<NameValuePair> secondaryFilters, EnumSet<Field> fields) {
if (limit == null) {
@ -76,8 +76,8 @@ public class MemoryApplicationTimelineStore
if (fields == null) {
fields = EnumSet.allOf(Field.class);
}
List<ATSEntity> entitiesSelected = new ArrayList<ATSEntity>();
for (ATSEntity entity : new PriorityQueue<ATSEntity>(entities.values())) {
List<TimelineEntity> entitiesSelected = new ArrayList<TimelineEntity>();
for (TimelineEntity entity : new PriorityQueue<TimelineEntity>(entities.values())) {
if (entitiesSelected.size() >= limit) {
break;
}
@ -91,7 +91,7 @@ public class MemoryApplicationTimelineStore
continue;
}
if (primaryFilter != null &&
!matchFilter(entity.getPrimaryFilters(), primaryFilter)) {
!matchPrimaryFilter(entity.getPrimaryFilters(), primaryFilter)) {
continue;
}
if (secondaryFilters != null) { // OR logic
@ -109,23 +109,23 @@ public class MemoryApplicationTimelineStore
}
entitiesSelected.add(entity);
}
List<ATSEntity> entitiesToReturn = new ArrayList<ATSEntity>();
for (ATSEntity entitySelected : entitiesSelected) {
List<TimelineEntity> entitiesToReturn = new ArrayList<TimelineEntity>();
for (TimelineEntity entitySelected : entitiesSelected) {
entitiesToReturn.add(maskFields(entitySelected, fields));
}
Collections.sort(entitiesToReturn);
ATSEntities entitiesWrapper = new ATSEntities();
TimelineEntities entitiesWrapper = new TimelineEntities();
entitiesWrapper.setEntities(entitiesToReturn);
return entitiesWrapper;
}
@Override
public ATSEntity getEntity(String entityId, String entityType,
public TimelineEntity getEntity(String entityId, String entityType,
EnumSet<Field> fieldsToRetrieve) {
if (fieldsToRetrieve == null) {
fieldsToRetrieve = EnumSet.allOf(Field.class);
}
ATSEntity entity = entities.get(new EntityIdentifier(entityId, entityType));
TimelineEntity entity = entities.get(new EntityIdentifier(entityId, entityType));
if (entity == null) {
return null;
} else {
@ -134,11 +134,11 @@ public class MemoryApplicationTimelineStore
}
@Override
public ATSEvents getEntityTimelines(String entityType,
public TimelineEvents getEntityTimelines(String entityType,
SortedSet<String> entityIds, Long limit, Long windowStart,
Long windowEnd,
Set<String> eventTypes) {
ATSEvents allEvents = new ATSEvents();
TimelineEvents allEvents = new TimelineEvents();
if (entityIds == null) {
return allEvents;
}
@ -153,14 +153,14 @@ public class MemoryApplicationTimelineStore
}
for (String entityId : entityIds) {
EntityIdentifier entityID = new EntityIdentifier(entityId, entityType);
ATSEntity entity = entities.get(entityID);
TimelineEntity entity = entities.get(entityID);
if (entity == null) {
continue;
}
ATSEventsOfOneEntity events = new ATSEventsOfOneEntity();
EventsOfOneEntity events = new EventsOfOneEntity();
events.setEntityId(entityId);
events.setEntityType(entityType);
for (ATSEvent event : entity.getEvents()) {
for (TimelineEvent event : entity.getEvents()) {
if (events.getEvents().size() >= limit) {
break;
}
@ -181,15 +181,15 @@ public class MemoryApplicationTimelineStore
}
@Override
public ATSPutErrors put(ATSEntities data) {
ATSPutErrors errors = new ATSPutErrors();
for (ATSEntity entity : data.getEntities()) {
public TimelinePutResponse put(TimelineEntities data) {
TimelinePutResponse response = new TimelinePutResponse();
for (TimelineEntity entity : data.getEntities()) {
EntityIdentifier entityId =
new EntityIdentifier(entity.getEntityId(), entity.getEntityType());
// store entity info in memory
ATSEntity existingEntity = entities.get(entityId);
TimelineEntity existingEntity = entities.get(entityId);
if (existingEntity == null) {
existingEntity = new ATSEntity();
existingEntity = new TimelineEntity();
existingEntity.setEntityId(entity.getEntityId());
existingEntity.setEntityType(entity.getEntityType());
existingEntity.setStartTime(entity.getStartTime());
@ -207,11 +207,11 @@ public class MemoryApplicationTimelineStore
if (existingEntity.getStartTime() == null) {
if (existingEntity.getEvents() == null
|| existingEntity.getEvents().isEmpty()) {
ATSPutError error = new ATSPutError();
TimelinePutError error = new TimelinePutError();
error.setEntityId(entityId.getId());
error.setEntityType(entityId.getType());
error.setErrorCode(ATSPutError.NO_START_TIME);
errors.addError(error);
error.setErrorCode(TimelinePutError.NO_START_TIME);
response.addError(error);
entities.remove(entityId);
continue;
} else {
@ -236,7 +236,7 @@ public class MemoryApplicationTimelineStore
if (entity.getRelatedEntities() == null) {
continue;
}
for (Map.Entry<String, List<String>> partRelatedEntities : entity
for (Map.Entry<String, Set<String>> partRelatedEntities : entity
.getRelatedEntities().entrySet()) {
if (partRelatedEntities == null) {
continue;
@ -244,12 +244,12 @@ public class MemoryApplicationTimelineStore
for (String idStr : partRelatedEntities.getValue()) {
EntityIdentifier relatedEntityId =
new EntityIdentifier(idStr, partRelatedEntities.getKey());
ATSEntity relatedEntity = entities.get(relatedEntityId);
TimelineEntity relatedEntity = entities.get(relatedEntityId);
if (relatedEntity != null) {
relatedEntity.addRelatedEntity(
existingEntity.getEntityType(), existingEntity.getEntityId());
} else {
relatedEntity = new ATSEntity();
relatedEntity = new TimelineEntity();
relatedEntity.setEntityId(relatedEntityId.getId());
relatedEntity.setEntityType(relatedEntityId.getType());
relatedEntity.setStartTime(existingEntity.getStartTime());
@ -260,13 +260,13 @@ public class MemoryApplicationTimelineStore
}
}
}
return errors;
return response;
}
private static ATSEntity maskFields(
ATSEntity entity, EnumSet<Field> fields) {
private static TimelineEntity maskFields(
TimelineEntity entity, EnumSet<Field> fields) {
// Conceal the fields that are not going to be exposed
ATSEntity entityToReturn = new ATSEntity();
TimelineEntity entityToReturn = new TimelineEntity();
entityToReturn.setEntityId(entity.getEntityId());
entityToReturn.setEntityType(entity.getEntityType());
entityToReturn.setStartTime(entity.getStartTime());
@ -293,4 +293,14 @@ public class MemoryApplicationTimelineStore
return true;
}
private static boolean matchPrimaryFilter(Map<String, Set<Object>> tags,
NameValuePair filter) {
Set<Object> value = tags.get(filter.getName());
if (value == null) { // doesn't have the filter
return false;
} else {
return value.contains(filter.getValue());
}
}
}

View File

@ -15,14 +15,14 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline;
package org.apache.hadoop.yarn.server.applicationhistoryservice.timeline;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* A class holding a name and value pair, used for specifying filters in
* {@link ApplicationTimelineReader}.
* {@link TimelineReader}.
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable

View File

@ -0,0 +1,141 @@
/**
* 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.hadoop.yarn.server.applicationhistoryservice.timeline;
import java.io.IOException;
import java.util.Collection;
import java.util.EnumSet;
import java.util.Set;
import java.util.SortedSet;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEvents;
/**
* This interface is for retrieving timeline information.
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
public interface TimelineReader {
/**
* Possible fields to retrieve for {@link #getEntities} and {@link #getEntity}
* .
*/
enum Field {
EVENTS,
RELATED_ENTITIES,
PRIMARY_FILTERS,
OTHER_INFO,
LAST_EVENT_ONLY
}
/**
* Default limit for {@link #getEntities} and {@link #getEntityTimelines}.
*/
final long DEFAULT_LIMIT = 100;
/**
* This method retrieves a list of entity information, {@link TimelineEntity}, sorted
* by the starting timestamp for the entity, descending.
*
* @param entityType
* The type of entities to return (required).
* @param limit
* A limit on the number of entities to return. If null, defaults to
* {@link #DEFAULT_LIMIT}.
* @param windowStart
* The earliest start timestamp to retrieve (exclusive). If null,
* defaults to retrieving all entities until the limit is reached.
* @param windowEnd
* The latest start timestamp to retrieve (inclusive). If null,
* defaults to {@link Long#MAX_VALUE}
* @param primaryFilter
* Retrieves only entities that have the specified primary filter. If
* null, retrieves all entities. This is an indexed retrieval, and no
* entities that do not match the filter are scanned.
* @param secondaryFilters
* Retrieves only entities that have exact matches for all the
* specified filters in their primary filters or other info. This is
* not an indexed retrieval, so all entities are scanned but only
* those matching the filters are returned.
* @param fieldsToRetrieve
* Specifies which fields of the entity object to retrieve (see
* {@link Field}). If the set of fields contains
* {@link Field#LAST_EVENT_ONLY} and not {@link Field#EVENTS}, the
* most recent event for each entity is retrieved. If null, retrieves
* all fields.
* @return An {@link TimelineEntities} object.
* @throws IOException
*/
TimelineEntities getEntities(String entityType,
Long limit, Long windowStart, Long windowEnd,
NameValuePair primaryFilter, Collection<NameValuePair> secondaryFilters,
EnumSet<Field> fieldsToRetrieve) throws IOException;
/**
* This method retrieves the entity information for a given entity.
*
* @param entityId
* The entity whose information will be retrieved.
* @param entityType
* The type of the entity.
* @param fieldsToRetrieve
* Specifies which fields of the entity object to retrieve (see
* {@link Field}). If the set of fields contains
* {@link Field#LAST_EVENT_ONLY} and not {@link Field#EVENTS}, the
* most recent event for each entity is retrieved. If null, retrieves
* all fields.
* @return An {@link TimelineEntity} object.
* @throws IOException
*/
TimelineEntity getEntity(String entityId, String entityType, EnumSet<Field>
fieldsToRetrieve) throws IOException;
/**
* This method retrieves the events for a list of entities all of the same
* entity type. The events for each entity are sorted in order of their
* timestamps, descending.
*
* @param entityType
* The type of entities to retrieve events for.
* @param entityIds
* The entity IDs to retrieve events for.
* @param limit
* A limit on the number of events to return for each entity. If
* null, defaults to {@link #DEFAULT_LIMIT} events per entity.
* @param windowStart
* If not null, retrieves only events later than the given time
* (exclusive)
* @param windowEnd
* If not null, retrieves only events earlier than the given time
* (inclusive)
* @param eventTypes
* Restricts the events returned to the given types. If null, events
* of all types will be returned.
* @return An {@link TimelineEvents} object.
* @throws IOException
*/
TimelineEvents getEntityTimelines(String entityType,
SortedSet<String> entityIds, Long limit, Long windowStart,
Long windowEnd, Set<String> eventTypes) throws IOException;
}

View File

@ -16,7 +16,7 @@
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline;
package org.apache.hadoop.yarn.server.applicationhistoryservice.timeline;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
@ -24,6 +24,6 @@ import org.apache.hadoop.service.Service;
@InterfaceAudience.Private
@InterfaceStability.Unstable
public interface ApplicationTimelineStore extends
Service, ApplicationTimelineReader, ApplicationTimelineWriter {
public interface TimelineStore extends
Service, TimelineReader, TimelineWriter {
}

View File

@ -16,31 +16,31 @@
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline;
package org.apache.hadoop.yarn.server.applicationhistoryservice.timeline;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSEntities;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSPutErrors;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
import java.io.IOException;
/**
* This interface is for storing application timeline information.
* This interface is for storing timeline information.
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
public interface ApplicationTimelineWriter {
public interface TimelineWriter {
/**
* Stores entity information to the application timeline store. Any errors
* occurring for individual put request objects will be reported in the
* response.
*
* @param data An {@link ATSEntities} object.
* @return An {@link ATSPutErrors} object.
* Stores entity information to the timeline store. Any errors occurring for
* individual put request objects will be reported in the response.
*
* @param data
* An {@link TimelineEntities} object.
* @return An {@link TimelinePutResponse} object.
* @throws IOException
*/
ATSPutErrors put(ATSEntities data) throws IOException;
TimelinePutResponse put(TimelineEntities data) throws IOException;
}

View File

@ -16,5 +16,5 @@
* limitations under the License.
*/
@InterfaceAudience.Private
package org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline;
package org.apache.hadoop.yarn.server.applicationhistoryservice.timeline;
import org.apache.hadoop.classification.InterfaceAudience;

View File

@ -21,7 +21,7 @@ import static org.apache.hadoop.yarn.util.StringHelper.pajoin;
import org.apache.hadoop.yarn.server.api.ApplicationContext;
import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryManager;
import org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline.ApplicationTimelineStore;
import org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.TimelineStore;
import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
import org.apache.hadoop.yarn.webapp.WebApp;
import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
@ -30,22 +30,22 @@ import org.apache.hadoop.yarn.webapp.YarnWebParams;
public class AHSWebApp extends WebApp implements YarnWebParams {
private final ApplicationHistoryManager applicationHistoryManager;
private final ApplicationTimelineStore applicationTimelineStore;
private final TimelineStore timelineStore;
public AHSWebApp(ApplicationHistoryManager applicationHistoryManager,
ApplicationTimelineStore applicationTimelineStore) {
TimelineStore timelineStore) {
this.applicationHistoryManager = applicationHistoryManager;
this.applicationTimelineStore = applicationTimelineStore;
this.timelineStore = timelineStore;
}
@Override
public void setup() {
bind(YarnJacksonJaxbJsonProvider.class);
bind(AHSWebServices.class);
bind(ATSWebServices.class);
bind(TimelineWebServices.class);
bind(GenericExceptionHandler.class);
bind(ApplicationContext.class).toInstance(applicationHistoryManager);
bind(ApplicationTimelineStore.class).toInstance(applicationTimelineStore);
bind(TimelineStore.class).toInstance(timelineStore);
route("/", AHSController.class);
route(pajoin("/apps", APP_STATE), AHSController.class);
route(pajoin("/app", APPLICATION_ID), AHSController.class, "app");

View File

@ -50,29 +50,29 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSEntities;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSEntity;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSEvents;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSPutErrors;
import org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline.ApplicationTimelineReader.Field;
import org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline.ApplicationTimelineStore;
import org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline.NameValuePair;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEvents;
import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
import org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.TimelineStore;
import org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.NameValuePair;
import org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.TimelineReader.Field;
import org.apache.hadoop.yarn.webapp.BadRequestException;
import com.google.inject.Inject;
import com.google.inject.Singleton;
@Singleton
@Path("/ws/v1/apptimeline")
@Path("/ws/v1/timeline")
//TODO: support XML serialization/deserialization
public class ATSWebServices {
public class TimelineWebServices {
private static final Log LOG = LogFactory.getLog(ATSWebServices.class);
private static final Log LOG = LogFactory.getLog(TimelineWebServices.class);
private ApplicationTimelineStore store;
private TimelineStore store;
@Inject
public ATSWebServices(ApplicationTimelineStore store) {
public TimelineWebServices(TimelineStore store) {
this.store = store;
}
@ -104,7 +104,7 @@ public class ATSWebServices {
}
/**
* Return the description of the application timeline web services.
* Return the description of the timeline web services.
*/
@GET
@Produces({ MediaType.APPLICATION_JSON /* , MediaType.APPLICATION_XML */})
@ -112,7 +112,7 @@ public class ATSWebServices {
@Context HttpServletRequest req,
@Context HttpServletResponse res) {
init(res);
return new AboutInfo("Application Timeline API");
return new AboutInfo("Timeline API");
}
/**
@ -121,7 +121,7 @@ public class ATSWebServices {
@GET
@Path("/{entityType}")
@Produces({ MediaType.APPLICATION_JSON /* , MediaType.APPLICATION_XML */})
public ATSEntities getEntities(
public TimelineEntities getEntities(
@Context HttpServletRequest req,
@Context HttpServletResponse res,
@PathParam("entityType") String entityType,
@ -132,7 +132,7 @@ public class ATSWebServices {
@QueryParam("limit") String limit,
@QueryParam("fields") String fields) {
init(res);
ATSEntities entities = null;
TimelineEntities entities = null;
try {
entities = store.getEntities(
parseStr(entityType),
@ -153,7 +153,7 @@ public class ATSWebServices {
Response.Status.INTERNAL_SERVER_ERROR);
}
if (entities == null) {
return new ATSEntities();
return new TimelineEntities();
}
return entities;
}
@ -164,14 +164,14 @@ public class ATSWebServices {
@GET
@Path("/{entityType}/{entityId}")
@Produces({ MediaType.APPLICATION_JSON /* , MediaType.APPLICATION_XML */})
public ATSEntity getEntity(
public TimelineEntity getEntity(
@Context HttpServletRequest req,
@Context HttpServletResponse res,
@PathParam("entityType") String entityType,
@PathParam("entityId") String entityId,
@QueryParam("fields") String fields) {
init(res);
ATSEntity entity = null;
TimelineEntity entity = null;
try {
entity =
store.getEntity(parseStr(entityId), parseStr(entityType),
@ -196,7 +196,7 @@ public class ATSWebServices {
@GET
@Path("/{entityType}/events")
@Produces({ MediaType.APPLICATION_JSON /* , MediaType.APPLICATION_XML */})
public ATSEvents getEvents(
public TimelineEvents getEvents(
@Context HttpServletRequest req,
@Context HttpServletResponse res,
@PathParam("entityType") String entityType,
@ -206,7 +206,7 @@ public class ATSWebServices {
@QueryParam("windowEnd") String windowEnd,
@QueryParam("limit") String limit) {
init(res);
ATSEvents events = null;
TimelineEvents events = null;
try {
events = store.getEntityTimelines(
parseStr(entityType),
@ -224,7 +224,7 @@ public class ATSWebServices {
Response.Status.INTERNAL_SERVER_ERROR);
}
if (events == null) {
return new ATSEvents();
return new TimelineEvents();
}
return events;
}
@ -235,13 +235,13 @@ public class ATSWebServices {
*/
@POST
@Consumes({ MediaType.APPLICATION_JSON /* , MediaType.APPLICATION_XML */})
public ATSPutErrors postEntities(
public TimelinePutResponse postEntities(
@Context HttpServletRequest req,
@Context HttpServletResponse res,
ATSEntities entities) {
TimelineEntities entities) {
init(res);
if (entities == null) {
return new ATSPutErrors();
return new TimelinePutResponse();
}
try {
return store.put(entities);

View File

@ -57,7 +57,7 @@ public class TestApplicationHistoryClientService extends
public void setup() {
historyServer = new ApplicationHistoryServer();
Configuration config = new YarnConfiguration();
config.setClass(YarnConfiguration.AHS_STORE,
config.setClass(YarnConfiguration.APPLICATION_HISTORY_STORE,
MemoryApplicationHistoryStore.class, ApplicationHistoryStore.class);
historyServer.init(config);
historyServer.start();

View File

@ -38,7 +38,7 @@ public class TestApplicationHistoryManagerImpl extends
@Before
public void setup() throws Exception {
Configuration config = new Configuration();
config.setClass(YarnConfiguration.AHS_STORE,
config.setClass(YarnConfiguration.APPLICATION_HISTORY_STORE,
MemoryApplicationHistoryStore.class, ApplicationHistoryStore.class);
applicationHistoryManagerImpl = new ApplicationHistoryManagerImpl();
applicationHistoryManagerImpl.init(config);

View File

@ -52,7 +52,7 @@ public class TestFileSystemApplicationHistoryStore extends
fs.initialize(new URI("/"), conf);
fsWorkingPath = new Path("Test");
fs.delete(fsWorkingPath, true);
conf.set(YarnConfiguration.FS_HISTORY_STORE_URI, fsWorkingPath.toString());
conf.set(YarnConfiguration.FS_APPLICATION_HISTORY_STORE_URI, fsWorkingPath.toString());
store = new FileSystemApplicationHistoryStore();
store.init(conf);
store.start();

View File

@ -15,11 +15,12 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline;
package org.apache.hadoop.yarn.server.applicationhistoryservice.timeline;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.io.WritableComparator;
import org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.GenericObjectMapper;
import org.junit.Test;
import java.io.IOException;

View File

@ -15,7 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline;
package org.apache.hadoop.yarn.server.applicationhistoryservice.timeline;
import java.io.File;
import java.io.IOException;
@ -25,21 +25,15 @@ import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileContext;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSEntities;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSEntity;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSPutErrors;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSPutErrors.ATSPutError;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import static org.junit.Assert.assertEquals;
@InterfaceAudience.Private
@InterfaceStability.Unstable
public class TestLeveldbApplicationTimelineStore
extends ApplicationTimelineStoreTestUtils {
public class TestLeveldbTimelineStore
extends TimelineStoreTestUtils {
private FileContext fsContext;
private File fsPath;
@ -50,9 +44,9 @@ public class TestLeveldbApplicationTimelineStore
fsPath = new File("target", this.getClass().getSimpleName() +
"-tmpDir").getAbsoluteFile();
fsContext.delete(new Path(fsPath.getAbsolutePath()), true);
conf.set(YarnConfiguration.ATS_LEVELDB_PATH_PROPERTY,
conf.set(YarnConfiguration.TIMELINE_SERVICE_LEVELDB_PATH,
fsPath.getAbsolutePath());
store = new LeveldbApplicationTimelineStore();
store = new LeveldbTimelineStore();
store.init(conf);
store.start();
loadTestData();
@ -68,7 +62,7 @@ public class TestLeveldbApplicationTimelineStore
@Test
public void testGetSingleEntity() throws IOException {
super.testGetSingleEntity();
((LeveldbApplicationTimelineStore)store).clearStartTimeCache();
((LeveldbTimelineStore)store).clearStartTimeCache();
super.testGetSingleEntity();
}

View File

@ -16,21 +16,23 @@
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline;
package org.apache.hadoop.yarn.server.applicationhistoryservice.timeline;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.TimelineStore;
import org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.MemoryTimelineStore;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import java.io.IOException;
public class TestMemoryApplicationTimelineStore
extends ApplicationTimelineStoreTestUtils {
public class TestMemoryTimelineStore
extends TimelineStoreTestUtils {
@Before
public void setup() throws Exception {
store = new MemoryApplicationTimelineStore();
store = new MemoryTimelineStore();
store.init(new YarnConfiguration());
store.start();
loadTestData();
@ -42,7 +44,7 @@ public class TestMemoryApplicationTimelineStore
store.stop();
}
public ApplicationTimelineStore getApplicationTimelineStore() {
public TimelineStore getTimelineStore() {
return store;
}

View File

@ -15,69 +15,80 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline;
package org.apache.hadoop.yarn.server.applicationhistoryservice.timeline;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import java.util.SortedSet;
import java.util.TreeSet;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSEntities;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSEntity;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSEvent;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSEvents.ATSEventsOfOneEntity;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSPutErrors;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSPutErrors.ATSPutError;
import org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline.ApplicationTimelineReader.Field;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEvents.EventsOfOneEntity;
import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse.TimelinePutError;
import org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.TimelineStore;
import org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.NameValuePair;
import org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.TimelineReader.Field;
public class ApplicationTimelineStoreTestUtils {
public class TimelineStoreTestUtils {
private static final Map<String, Object> EMPTY_MAP = Collections.emptyMap();
private static final Map<String, List<String>> EMPTY_REL_ENTITIES =
new HashMap<String, List<String>>();
protected static final Map<String, Object> EMPTY_MAP =
Collections.emptyMap();
protected static final Map<String, Set<Object>> EMPTY_PRIMARY_FILTERS =
Collections.emptyMap();
protected static final Map<String, Set<String>> EMPTY_REL_ENTITIES =
Collections.emptyMap();
protected ApplicationTimelineStore store;
private String entity1;
private String entityType1;
private String entity1b;
private String entity2;
private String entityType2;
private Map<String, Object> primaryFilters;
private Map<String, Object> secondaryFilters;
private Map<String, Object> allFilters;
private Map<String, Object> otherInfo;
private Map<String, List<String>> relEntityMap;
private NameValuePair userFilter;
private Collection<NameValuePair> goodTestingFilters;
private Collection<NameValuePair> badTestingFilters;
private ATSEvent ev1;
private ATSEvent ev2;
private ATSEvent ev3;
private ATSEvent ev4;
private Map<String, Object> eventInfo;
private List<ATSEvent> events1;
private List<ATSEvent> events2;
protected TimelineStore store;
protected String entityId1;
protected String entityType1;
protected String entityId1b;
protected String entityId2;
protected String entityType2;
protected Map<String, Set<Object>> primaryFilters;
protected Map<String, Object> secondaryFilters;
protected Map<String, Object> allFilters;
protected Map<String, Object> otherInfo;
protected Map<String, Set<String>> relEntityMap;
protected NameValuePair userFilter;
protected Collection<NameValuePair> goodTestingFilters;
protected Collection<NameValuePair> badTestingFilters;
protected TimelineEvent ev1;
protected TimelineEvent ev2;
protected TimelineEvent ev3;
protected TimelineEvent ev4;
protected Map<String, Object> eventInfo;
protected List<TimelineEvent> events1;
protected List<TimelineEvent> events2;
/**
* Load test data into the given store
*/
protected void loadTestData() throws IOException {
ATSEntities atsEntities = new ATSEntities();
Map<String, Object> primaryFilters = new HashMap<String, Object>();
primaryFilters.put("user", "username");
primaryFilters.put("appname", 12345l);
TimelineEntities entities = new TimelineEntities();
Map<String, Set<Object>> primaryFilters =
new HashMap<String, Set<Object>>();
Set<Object> l1 = new HashSet<Object>();
l1.add("username");
Set<Object> l2 = new HashSet<Object>();
l2.add(12345l);
primaryFilters.put("user", l1);
primaryFilters.put("appname", l2);
Map<String, Object> secondaryFilters = new HashMap<String, Object>();
secondaryFilters.put("startTime", 123456l);
secondaryFilters.put("status", "RUNNING");
@ -85,62 +96,62 @@ public class ApplicationTimelineStoreTestUtils {
otherInfo1.put("info1", "val1");
otherInfo1.putAll(secondaryFilters);
String entity1 = "id_1";
String entityId1 = "id_1";
String entityType1 = "type_1";
String entity1b = "id_2";
String entity2 = "id_2";
String entityId1b = "id_2";
String entityId2 = "id_2";
String entityType2 = "type_2";
Map<String, List<String>> relatedEntities =
new HashMap<String, List<String>>();
relatedEntities.put(entityType2, Collections.singletonList(entity2));
Map<String, Set<String>> relatedEntities =
new HashMap<String, Set<String>>();
relatedEntities.put(entityType2, Collections.singleton(entityId2));
ATSEvent ev3 = createEvent(789l, "launch_event", null);
ATSEvent ev4 = createEvent(-123l, "init_event", null);
List<ATSEvent> events = new ArrayList<ATSEvent>();
TimelineEvent ev3 = createEvent(789l, "launch_event", null);
TimelineEvent ev4 = createEvent(-123l, "init_event", null);
List<TimelineEvent> events = new ArrayList<TimelineEvent>();
events.add(ev3);
events.add(ev4);
atsEntities.setEntities(Collections.singletonList(createEntity(entity2,
entities.setEntities(Collections.singletonList(createEntity(entityId2,
entityType2, null, events, null, null, null)));
ATSPutErrors response = store.put(atsEntities);
TimelinePutResponse response = store.put(entities);
assertEquals(0, response.getErrors().size());
ATSEvent ev1 = createEvent(123l, "start_event", null);
atsEntities.setEntities(Collections.singletonList(createEntity(entity1,
TimelineEvent ev1 = createEvent(123l, "start_event", null);
entities.setEntities(Collections.singletonList(createEntity(entityId1,
entityType1, 123l, Collections.singletonList(ev1),
relatedEntities, primaryFilters, otherInfo1)));
response = store.put(atsEntities);
response = store.put(entities);
assertEquals(0, response.getErrors().size());
atsEntities.setEntities(Collections.singletonList(createEntity(entity1b,
entities.setEntities(Collections.singletonList(createEntity(entityId1b,
entityType1, null, Collections.singletonList(ev1), relatedEntities,
primaryFilters, otherInfo1)));
response = store.put(atsEntities);
response = store.put(entities);
assertEquals(0, response.getErrors().size());
Map<String, Object> eventInfo = new HashMap<String, Object>();
eventInfo.put("event info 1", "val1");
ATSEvent ev2 = createEvent(456l, "end_event", eventInfo);
TimelineEvent ev2 = createEvent(456l, "end_event", eventInfo);
Map<String, Object> otherInfo2 = new HashMap<String, Object>();
otherInfo2.put("info2", "val2");
atsEntities.setEntities(Collections.singletonList(createEntity(entity1,
entities.setEntities(Collections.singletonList(createEntity(entityId1,
entityType1, null, Collections.singletonList(ev2), null,
primaryFilters, otherInfo2)));
response = store.put(atsEntities);
response = store.put(entities);
assertEquals(0, response.getErrors().size());
atsEntities.setEntities(Collections.singletonList(createEntity(entity1b,
entities.setEntities(Collections.singletonList(createEntity(entityId1b,
entityType1, 789l, Collections.singletonList(ev2), null,
primaryFilters, otherInfo2)));
response = store.put(atsEntities);
response = store.put(entities);
assertEquals(0, response.getErrors().size());
atsEntities.setEntities(Collections.singletonList(createEntity(
entities.setEntities(Collections.singletonList(createEntity(
"badentityid", "badentity", null, null, null, null, otherInfo1)));
response = store.put(atsEntities);
response = store.put(entities);
assertEquals(1, response.getErrors().size());
ATSPutError error = response.getErrors().get(0);
TimelinePutError error = response.getErrors().get(0);
assertEquals("badentityid", error.getEntityId());
assertEquals("badentity", error.getEntityType());
assertEquals(ATSPutError.NO_START_TIME, error.getErrorCode());
assertEquals(TimelinePutError.NO_START_TIME, error.getErrorCode());
}
/**
@ -156,24 +167,32 @@ public class ApplicationTimelineStoreTestUtils {
badTestingFilters.add(new NameValuePair("appname", 12345l));
badTestingFilters.add(new NameValuePair("status", "FINISHED"));
primaryFilters = new HashMap<String, Object>();
primaryFilters.put("user", "username");
primaryFilters.put("appname", 12345l);
primaryFilters = new HashMap<String, Set<Object>>();
Set<Object> l1 = new HashSet<Object>();
l1.add("username");
Set<Object> l2 = new HashSet<Object>();
l2.add(12345l);
primaryFilters.put("user", l1);
primaryFilters.put("appname", l2);
secondaryFilters = new HashMap<String, Object>();
secondaryFilters.put("startTime", 123456l);
secondaryFilters.put("status", "RUNNING");
allFilters = new HashMap<String, Object>();
allFilters.putAll(secondaryFilters);
allFilters.putAll(primaryFilters);
for (Entry<String, Set<Object>> pf : primaryFilters.entrySet()) {
for (Object o : pf.getValue()) {
allFilters.put(pf.getKey(), o);
}
}
otherInfo = new HashMap<String, Object>();
otherInfo.put("info1", "val1");
otherInfo.put("info2", "val2");
otherInfo.putAll(secondaryFilters);
entity1 = "id_1";
entityId1 = "id_1";
entityType1 = "type_1";
entity1b = "id_2";
entity2 = "id_2";
entityId1b = "id_2";
entityId2 = "id_2";
entityType2 = "type_2";
ev1 = createEvent(123l, "start_event", null);
@ -181,20 +200,20 @@ public class ApplicationTimelineStoreTestUtils {
eventInfo = new HashMap<String, Object>();
eventInfo.put("event info 1", "val1");
ev2 = createEvent(456l, "end_event", eventInfo);
events1 = new ArrayList<ATSEvent>();
events1 = new ArrayList<TimelineEvent>();
events1.add(ev2);
events1.add(ev1);
relEntityMap =
new HashMap<String, List<String>>();
List<String> ids = new ArrayList<String>();
ids.add(entity1);
ids.add(entity1b);
new HashMap<String, Set<String>>();
Set<String> ids = new HashSet<String>();
ids.add(entityId1);
ids.add(entityId1b);
relEntityMap.put(entityType1, ids);
ev3 = createEvent(789l, "launch_event", null);
ev4 = createEvent(-123l, "init_event", null);
events2 = new ArrayList<ATSEvent>();
events2 = new ArrayList<TimelineEvent>();
events2.add(ev3);
events2.add(ev4);
}
@ -204,39 +223,39 @@ public class ApplicationTimelineStoreTestUtils {
verifyEntityInfo(null, null, null, null, null, null,
store.getEntity("id_1", "type_2", EnumSet.allOf(Field.class)));
verifyEntityInfo(entity1, entityType1, events1, EMPTY_REL_ENTITIES,
primaryFilters, otherInfo, store.getEntity(entity1, entityType1,
verifyEntityInfo(entityId1, entityType1, events1, EMPTY_REL_ENTITIES,
primaryFilters, otherInfo, store.getEntity(entityId1, entityType1,
EnumSet.allOf(Field.class)));
verifyEntityInfo(entity1b, entityType1, events1, EMPTY_REL_ENTITIES,
primaryFilters, otherInfo, store.getEntity(entity1b, entityType1,
verifyEntityInfo(entityId1b, entityType1, events1, EMPTY_REL_ENTITIES,
primaryFilters, otherInfo, store.getEntity(entityId1b, entityType1,
EnumSet.allOf(Field.class)));
verifyEntityInfo(entity2, entityType2, events2, relEntityMap, EMPTY_MAP,
EMPTY_MAP, store.getEntity(entity2, entityType2,
verifyEntityInfo(entityId2, entityType2, events2, relEntityMap,
EMPTY_PRIMARY_FILTERS, EMPTY_MAP, store.getEntity(entityId2, entityType2,
EnumSet.allOf(Field.class)));
// test getting single fields
verifyEntityInfo(entity1, entityType1, events1, null, null, null,
store.getEntity(entity1, entityType1, EnumSet.of(Field.EVENTS)));
verifyEntityInfo(entityId1, entityType1, events1, null, null, null,
store.getEntity(entityId1, entityType1, EnumSet.of(Field.EVENTS)));
verifyEntityInfo(entity1, entityType1, Collections.singletonList(ev2),
null, null, null, store.getEntity(entity1, entityType1,
verifyEntityInfo(entityId1, entityType1, Collections.singletonList(ev2),
null, null, null, store.getEntity(entityId1, entityType1,
EnumSet.of(Field.LAST_EVENT_ONLY)));
verifyEntityInfo(entity1b, entityType1, events1, EMPTY_REL_ENTITIES,
primaryFilters, otherInfo, store.getEntity(entity1b, entityType1,
verifyEntityInfo(entityId1b, entityType1, events1, EMPTY_REL_ENTITIES,
primaryFilters, otherInfo, store.getEntity(entityId1b, entityType1,
null));
verifyEntityInfo(entity1, entityType1, null, null, primaryFilters, null,
store.getEntity(entity1, entityType1,
verifyEntityInfo(entityId1, entityType1, null, null, primaryFilters, null,
store.getEntity(entityId1, entityType1,
EnumSet.of(Field.PRIMARY_FILTERS)));
verifyEntityInfo(entity1, entityType1, null, null, null, otherInfo,
store.getEntity(entity1, entityType1, EnumSet.of(Field.OTHER_INFO)));
verifyEntityInfo(entityId1, entityType1, null, null, null, otherInfo,
store.getEntity(entityId1, entityType1, EnumSet.of(Field.OTHER_INFO)));
verifyEntityInfo(entity2, entityType2, null, relEntityMap, null, null,
store.getEntity(entity2, entityType2,
verifyEntityInfo(entityId2, entityType2, null, relEntityMap, null, null,
store.getEntity(entityId2, entityType2,
EnumSet.of(Field.RELATED_ENTITIES)));
}
@ -255,31 +274,31 @@ public class ApplicationTimelineStoreTestUtils {
store.getEntities("type_3", null, null, null, userFilter,
null, null).getEntities().size());
List<ATSEntity> entities =
List<TimelineEntity> entities =
store.getEntities("type_1", null, null, null, null, null,
EnumSet.allOf(Field.class)).getEntities();
assertEquals(2, entities.size());
verifyEntityInfo(entity1, entityType1, events1, EMPTY_REL_ENTITIES,
verifyEntityInfo(entityId1, entityType1, events1, EMPTY_REL_ENTITIES,
primaryFilters, otherInfo, entities.get(0));
verifyEntityInfo(entity1b, entityType1, events1, EMPTY_REL_ENTITIES,
verifyEntityInfo(entityId1b, entityType1, events1, EMPTY_REL_ENTITIES,
primaryFilters, otherInfo, entities.get(1));
entities = store.getEntities("type_2", null, null, null, null, null,
EnumSet.allOf(Field.class)).getEntities();
assertEquals(1, entities.size());
verifyEntityInfo(entity2, entityType2, events2, relEntityMap, EMPTY_MAP,
EMPTY_MAP, entities.get(0));
verifyEntityInfo(entityId2, entityType2, events2, relEntityMap,
EMPTY_PRIMARY_FILTERS, EMPTY_MAP, entities.get(0));
entities = store.getEntities("type_1", 1l, null, null, null, null,
EnumSet.allOf(Field.class)).getEntities();
assertEquals(1, entities.size());
verifyEntityInfo(entity1, entityType1, events1, EMPTY_REL_ENTITIES,
verifyEntityInfo(entityId1, entityType1, events1, EMPTY_REL_ENTITIES,
primaryFilters, otherInfo, entities.get(0));
entities = store.getEntities("type_1", 1l, 0l, null, null, null,
EnumSet.allOf(Field.class)).getEntities();
assertEquals(1, entities.size());
verifyEntityInfo(entity1, entityType1, events1, EMPTY_REL_ENTITIES,
verifyEntityInfo(entityId1, entityType1, events1, EMPTY_REL_ENTITIES,
primaryFilters, otherInfo, entities.get(0));
entities = store.getEntities("type_1", null, 234l, null, null, null,
@ -297,17 +316,17 @@ public class ApplicationTimelineStoreTestUtils {
entities = store.getEntities("type_1", null, null, 345l, null, null,
EnumSet.allOf(Field.class)).getEntities();
assertEquals(2, entities.size());
verifyEntityInfo(entity1, entityType1, events1, EMPTY_REL_ENTITIES,
verifyEntityInfo(entityId1, entityType1, events1, EMPTY_REL_ENTITIES,
primaryFilters, otherInfo, entities.get(0));
verifyEntityInfo(entity1b, entityType1, events1, EMPTY_REL_ENTITIES,
verifyEntityInfo(entityId1b, entityType1, events1, EMPTY_REL_ENTITIES,
primaryFilters, otherInfo, entities.get(1));
entities = store.getEntities("type_1", null, null, 123l, null, null,
EnumSet.allOf(Field.class)).getEntities();
assertEquals(2, entities.size());
verifyEntityInfo(entity1, entityType1, events1, EMPTY_REL_ENTITIES,
verifyEntityInfo(entityId1, entityType1, events1, EMPTY_REL_ENTITIES,
primaryFilters, otherInfo, entities.get(0));
verifyEntityInfo(entity1b, entityType1, events1, EMPTY_REL_ENTITIES,
verifyEntityInfo(entityId1b, entityType1, events1, EMPTY_REL_ENTITIES,
primaryFilters, otherInfo, entities.get(1));
}
@ -326,12 +345,12 @@ public class ApplicationTimelineStoreTestUtils {
new NameValuePair("none", "none"), null,
EnumSet.allOf(Field.class)).getEntities().size());
List<ATSEntity> entities = store.getEntities("type_1", null, null, null,
List<TimelineEntity> entities = store.getEntities("type_1", null, null, null,
userFilter, null, EnumSet.allOf(Field.class)).getEntities();
assertEquals(2, entities.size());
verifyEntityInfo(entity1, entityType1, events1, EMPTY_REL_ENTITIES,
verifyEntityInfo(entityId1, entityType1, events1, EMPTY_REL_ENTITIES,
primaryFilters, otherInfo, entities.get(0));
verifyEntityInfo(entity1b, entityType1, events1, EMPTY_REL_ENTITIES,
verifyEntityInfo(entityId1b, entityType1, events1, EMPTY_REL_ENTITIES,
primaryFilters, otherInfo, entities.get(1));
entities = store.getEntities("type_2", null, null, null, userFilter, null,
@ -341,13 +360,13 @@ public class ApplicationTimelineStoreTestUtils {
entities = store.getEntities("type_1", 1l, null, null, userFilter, null,
EnumSet.allOf(Field.class)).getEntities();
assertEquals(1, entities.size());
verifyEntityInfo(entity1, entityType1, events1, EMPTY_REL_ENTITIES,
verifyEntityInfo(entityId1, entityType1, events1, EMPTY_REL_ENTITIES,
primaryFilters, otherInfo, entities.get(0));
entities = store.getEntities("type_1", 1l, 0l, null, userFilter, null,
EnumSet.allOf(Field.class)).getEntities();
assertEquals(1, entities.size());
verifyEntityInfo(entity1, entityType1, events1, EMPTY_REL_ENTITIES,
verifyEntityInfo(entityId1, entityType1, events1, EMPTY_REL_ENTITIES,
primaryFilters, otherInfo, entities.get(0));
entities = store.getEntities("type_1", null, 234l, null, userFilter, null,
@ -361,28 +380,28 @@ public class ApplicationTimelineStoreTestUtils {
entities = store.getEntities("type_1", null, null, 345l, userFilter, null,
EnumSet.allOf(Field.class)).getEntities();
assertEquals(2, entities.size());
verifyEntityInfo(entity1, entityType1, events1, EMPTY_REL_ENTITIES,
verifyEntityInfo(entityId1, entityType1, events1, EMPTY_REL_ENTITIES,
primaryFilters, otherInfo, entities.get(0));
verifyEntityInfo(entity1b, entityType1, events1, EMPTY_REL_ENTITIES,
verifyEntityInfo(entityId1b, entityType1, events1, EMPTY_REL_ENTITIES,
primaryFilters, otherInfo, entities.get(1));
}
public void testGetEntitiesWithSecondaryFilters() throws IOException {
// test using secondary filter
List<ATSEntity> entities = store.getEntities("type_1", null, null, null,
List<TimelineEntity> entities = store.getEntities("type_1", null, null, null,
null, goodTestingFilters, EnumSet.allOf(Field.class)).getEntities();
assertEquals(2, entities.size());
verifyEntityInfo(entity1, entityType1, events1, EMPTY_REL_ENTITIES,
verifyEntityInfo(entityId1, entityType1, events1, EMPTY_REL_ENTITIES,
primaryFilters, otherInfo, entities.get(0));
verifyEntityInfo(entity1b, entityType1, events1, EMPTY_REL_ENTITIES,
verifyEntityInfo(entityId1b, entityType1, events1, EMPTY_REL_ENTITIES,
primaryFilters, otherInfo, entities.get(1));
entities = store.getEntities("type_1", null, null, null, userFilter,
goodTestingFilters, EnumSet.allOf(Field.class)).getEntities();
assertEquals(2, entities.size());
verifyEntityInfo(entity1, entityType1, events1, EMPTY_REL_ENTITIES,
verifyEntityInfo(entityId1, entityType1, events1, EMPTY_REL_ENTITIES,
primaryFilters, otherInfo, entities.get(0));
verifyEntityInfo(entity1b, entityType1, events1, EMPTY_REL_ENTITIES,
verifyEntityInfo(entityId1b, entityType1, events1, EMPTY_REL_ENTITIES,
primaryFilters, otherInfo, entities.get(1));
entities = store.getEntities("type_1", null, null, null, null,
@ -397,102 +416,106 @@ public class ApplicationTimelineStoreTestUtils {
public void testGetEvents() throws IOException {
// test getting entity timelines
SortedSet<String> sortedSet = new TreeSet<String>();
sortedSet.add(entity1);
List<ATSEventsOfOneEntity> timelines =
sortedSet.add(entityId1);
List<EventsOfOneEntity> timelines =
store.getEntityTimelines(entityType1, sortedSet, null, null,
null, null).getAllEvents();
assertEquals(1, timelines.size());
verifyEntityTimeline(timelines.get(0), entity1, entityType1, ev2, ev1);
verifyEntityTimeline(timelines.get(0), entityId1, entityType1, ev2, ev1);
sortedSet.add(entity1b);
sortedSet.add(entityId1b);
timelines = store.getEntityTimelines(entityType1, sortedSet, null,
null, null, null).getAllEvents();
assertEquals(2, timelines.size());
verifyEntityTimeline(timelines.get(0), entity1, entityType1, ev2, ev1);
verifyEntityTimeline(timelines.get(1), entity1b, entityType1, ev2, ev1);
verifyEntityTimeline(timelines.get(0), entityId1, entityType1, ev2, ev1);
verifyEntityTimeline(timelines.get(1), entityId1b, entityType1, ev2, ev1);
timelines = store.getEntityTimelines(entityType1, sortedSet, 1l,
null, null, null).getAllEvents();
assertEquals(2, timelines.size());
verifyEntityTimeline(timelines.get(0), entity1, entityType1, ev2);
verifyEntityTimeline(timelines.get(1), entity1b, entityType1, ev2);
verifyEntityTimeline(timelines.get(0), entityId1, entityType1, ev2);
verifyEntityTimeline(timelines.get(1), entityId1b, entityType1, ev2);
timelines = store.getEntityTimelines(entityType1, sortedSet, null,
345l, null, null).getAllEvents();
assertEquals(2, timelines.size());
verifyEntityTimeline(timelines.get(0), entity1, entityType1, ev2);
verifyEntityTimeline(timelines.get(1), entity1b, entityType1, ev2);
verifyEntityTimeline(timelines.get(0), entityId1, entityType1, ev2);
verifyEntityTimeline(timelines.get(1), entityId1b, entityType1, ev2);
timelines = store.getEntityTimelines(entityType1, sortedSet, null,
123l, null, null).getAllEvents();
assertEquals(2, timelines.size());
verifyEntityTimeline(timelines.get(0), entity1, entityType1, ev2);
verifyEntityTimeline(timelines.get(1), entity1b, entityType1, ev2);
verifyEntityTimeline(timelines.get(0), entityId1, entityType1, ev2);
verifyEntityTimeline(timelines.get(1), entityId1b, entityType1, ev2);
timelines = store.getEntityTimelines(entityType1, sortedSet, null,
null, 345l, null).getAllEvents();
assertEquals(2, timelines.size());
verifyEntityTimeline(timelines.get(0), entity1, entityType1, ev1);
verifyEntityTimeline(timelines.get(1), entity1b, entityType1, ev1);
verifyEntityTimeline(timelines.get(0), entityId1, entityType1, ev1);
verifyEntityTimeline(timelines.get(1), entityId1b, entityType1, ev1);
timelines = store.getEntityTimelines(entityType1, sortedSet, null,
null, 123l, null).getAllEvents();
assertEquals(2, timelines.size());
verifyEntityTimeline(timelines.get(0), entity1, entityType1, ev1);
verifyEntityTimeline(timelines.get(1), entity1b, entityType1, ev1);
verifyEntityTimeline(timelines.get(0), entityId1, entityType1, ev1);
verifyEntityTimeline(timelines.get(1), entityId1b, entityType1, ev1);
timelines = store.getEntityTimelines(entityType1, sortedSet, null,
null, null, Collections.singleton("end_event")).getAllEvents();
assertEquals(2, timelines.size());
verifyEntityTimeline(timelines.get(0), entity1, entityType1, ev2);
verifyEntityTimeline(timelines.get(1), entity1b, entityType1, ev2);
verifyEntityTimeline(timelines.get(0), entityId1, entityType1, ev2);
verifyEntityTimeline(timelines.get(1), entityId1b, entityType1, ev2);
sortedSet.add(entity2);
sortedSet.add(entityId2);
timelines = store.getEntityTimelines(entityType2, sortedSet, null,
null, null, null).getAllEvents();
assertEquals(1, timelines.size());
verifyEntityTimeline(timelines.get(0), entity2, entityType2, ev3, ev4);
verifyEntityTimeline(timelines.get(0), entityId2, entityType2, ev3, ev4);
}
/**
* Verify a single entity
*/
private static void verifyEntityInfo(String entity, String entityType,
List<ATSEvent> events, Map<String, List<String>> relatedEntities,
Map<String, Object> primaryFilters, Map<String, Object> otherInfo,
ATSEntity retrievedEntityInfo) {
if (entity == null) {
protected static void verifyEntityInfo(String entityId, String entityType,
List<TimelineEvent> events, Map<String, Set<String>> relatedEntities,
Map<String, Set<Object>> primaryFilters, Map<String, Object> otherInfo,
TimelineEntity retrievedEntityInfo) {
if (entityId == null) {
assertNull(retrievedEntityInfo);
return;
}
assertEquals(entity, retrievedEntityInfo.getEntityId());
assertEquals(entityId, retrievedEntityInfo.getEntityId());
assertEquals(entityType, retrievedEntityInfo.getEntityType());
if (events == null)
if (events == null) {
assertNull(retrievedEntityInfo.getEvents());
else
} else {
assertEquals(events, retrievedEntityInfo.getEvents());
if (relatedEntities == null)
}
if (relatedEntities == null) {
assertNull(retrievedEntityInfo.getRelatedEntities());
else
} else {
assertEquals(relatedEntities, retrievedEntityInfo.getRelatedEntities());
if (primaryFilters == null)
}
if (primaryFilters == null) {
assertNull(retrievedEntityInfo.getPrimaryFilters());
else
} else {
assertTrue(primaryFilters.equals(
retrievedEntityInfo.getPrimaryFilters()));
if (otherInfo == null)
}
if (otherInfo == null) {
assertNull(retrievedEntityInfo.getOtherInfo());
else
} else {
assertTrue(otherInfo.equals(retrievedEntityInfo.getOtherInfo()));
}
}
/**
* Verify timeline events
*/
private static void verifyEntityTimeline(
ATSEventsOfOneEntity retrievedEvents, String entity, String entityType,
ATSEvent... actualEvents) {
assertEquals(entity, retrievedEvents.getEntityId());
EventsOfOneEntity retrievedEvents, String entityId, String entityType,
TimelineEvent... actualEvents) {
assertEquals(entityId, retrievedEvents.getEntityId());
assertEquals(entityType, retrievedEvents.getEntityType());
assertEquals(actualEvents.length, retrievedEvents.getEvents().size());
for (int i = 0; i < actualEvents.length; i++) {
@ -503,32 +526,36 @@ public class ApplicationTimelineStoreTestUtils {
/**
* Create a test entity
*/
private static ATSEntity createEntity(String entity, String entityType,
Long startTime, List<ATSEvent> events,
Map<String, List<String>> relatedEntities,
Map<String, Object> primaryFilters, Map<String, Object> otherInfo) {
ATSEntity atsEntity = new ATSEntity();
atsEntity.setEntityId(entity);
atsEntity.setEntityType(entityType);
atsEntity.setStartTime(startTime);
atsEntity.setEvents(events);
if (relatedEntities != null)
for (Entry<String, List<String>> e : relatedEntities.entrySet())
for (String v : e.getValue())
atsEntity.addRelatedEntity(e.getKey(), v);
else
atsEntity.setRelatedEntities(null);
atsEntity.setPrimaryFilters(primaryFilters);
atsEntity.setOtherInfo(otherInfo);
return atsEntity;
protected static TimelineEntity createEntity(String entityId, String entityType,
Long startTime, List<TimelineEvent> events,
Map<String, Set<String>> relatedEntities,
Map<String, Set<Object>> primaryFilters,
Map<String, Object> otherInfo) {
TimelineEntity entity = new TimelineEntity();
entity.setEntityId(entityId);
entity.setEntityType(entityType);
entity.setStartTime(startTime);
entity.setEvents(events);
if (relatedEntities != null) {
for (Entry<String, Set<String>> e : relatedEntities.entrySet()) {
for (String v : e.getValue()) {
entity.addRelatedEntity(e.getKey(), v);
}
}
} else {
entity.setRelatedEntities(null);
}
entity.setPrimaryFilters(primaryFilters);
entity.setOtherInfo(otherInfo);
return entity;
}
/**
* Create a test event
*/
private static ATSEvent createEvent(long timestamp, String type, Map<String,
private static TimelineEvent createEvent(long timestamp, String type, Map<String,
Object> info) {
ATSEvent event = new ATSEvent();
TimelineEvent event = new TimelineEvent();
event.setTimestamp(timestamp);
event.setEventType(type);
event.setEventInfo(info);

View File

@ -24,13 +24,13 @@ import javax.ws.rs.core.MediaType;
import junit.framework.Assert;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSEntities;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSEntity;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSEvent;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSEvents;
import org.apache.hadoop.yarn.api.records.apptimeline.ATSPutErrors;
import org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline.ApplicationTimelineStore;
import org.apache.hadoop.yarn.server.applicationhistoryservice.apptimeline.TestMemoryApplicationTimelineStore;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEvents;
import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
import org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.TimelineStore;
import org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.TestMemoryTimelineStore;
import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
import org.junit.Test;
@ -47,23 +47,23 @@ import com.sun.jersey.test.framework.JerseyTest;
import com.sun.jersey.test.framework.WebAppDescriptor;
public class TestATSWebServices extends JerseyTest {
public class TestTimelineWebServices extends JerseyTest {
private static ApplicationTimelineStore store;
private static TimelineStore store;
private Injector injector = Guice.createInjector(new ServletModule() {
@Override
protected void configureServlets() {
bind(YarnJacksonJaxbJsonProvider.class);
bind(ATSWebServices.class);
bind(TimelineWebServices.class);
bind(GenericExceptionHandler.class);
try{
store = mockApplicationTimelineStore();
store = mockTimelineStore();
} catch (Exception e) {
Assert.fail();
}
bind(ApplicationTimelineStore.class).toInstance(store);
bind(TimelineStore.class).toInstance(store);
serve("/*").with(GuiceContainer.class);
}
@ -77,15 +77,15 @@ public class TestATSWebServices extends JerseyTest {
}
}
private ApplicationTimelineStore mockApplicationTimelineStore()
private TimelineStore mockTimelineStore()
throws Exception {
TestMemoryApplicationTimelineStore store =
new TestMemoryApplicationTimelineStore();
TestMemoryTimelineStore store =
new TestMemoryTimelineStore();
store.setup();
return store.getApplicationTimelineStore();
return store.getTimelineStore();
}
public TestATSWebServices() {
public TestTimelineWebServices() {
super(new WebAppDescriptor.Builder(
"org.apache.hadoop.yarn.server.applicationhistoryservice.webapp")
.contextListenerClass(GuiceServletConfig.class)
@ -99,28 +99,28 @@ public class TestATSWebServices extends JerseyTest {
@Test
public void testAbout() throws Exception {
WebResource r = resource();
ClientResponse response = r.path("ws").path("v1").path("apptimeline")
ClientResponse response = r.path("ws").path("v1").path("timeline")
.accept(MediaType.APPLICATION_JSON)
.get(ClientResponse.class);
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
ATSWebServices.AboutInfo about =
response.getEntity(ATSWebServices.AboutInfo.class);
TimelineWebServices.AboutInfo about =
response.getEntity(TimelineWebServices.AboutInfo.class);
Assert.assertNotNull(about);
Assert.assertEquals("Application Timeline API", about.getAbout());
Assert.assertEquals("Timeline API", about.getAbout());
}
@Test
public void testGetEntities() throws Exception {
WebResource r = resource();
ClientResponse response = r.path("ws").path("v1").path("apptimeline")
ClientResponse response = r.path("ws").path("v1").path("timeline")
.path("type_1")
.accept(MediaType.APPLICATION_JSON)
.get(ClientResponse.class);
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
ATSEntities entities = response.getEntity(ATSEntities.class);
TimelineEntities entities = response.getEntity(TimelineEntities.class);
Assert.assertNotNull(entities);
Assert.assertEquals(2, entities.getEntities().size());
ATSEntity entity1 = entities.getEntities().get(0);
TimelineEntity entity1 = entities.getEntities().get(0);
Assert.assertNotNull(entity1);
Assert.assertEquals("id_1", entity1.getEntityId());
Assert.assertEquals("type_1", entity1.getEntityType());
@ -128,7 +128,7 @@ public class TestATSWebServices extends JerseyTest {
Assert.assertEquals(2, entity1.getEvents().size());
Assert.assertEquals(2, entity1.getPrimaryFilters().size());
Assert.assertEquals(4, entity1.getOtherInfo().size());
ATSEntity entity2 = entities.getEntities().get(1);
TimelineEntity entity2 = entities.getEntities().get(1);
Assert.assertNotNull(entity2);
Assert.assertEquals("id_2", entity2.getEntityId());
Assert.assertEquals("type_1", entity2.getEntityType());
@ -141,12 +141,12 @@ public class TestATSWebServices extends JerseyTest {
@Test
public void testGetEntity() throws Exception {
WebResource r = resource();
ClientResponse response = r.path("ws").path("v1").path("apptimeline")
ClientResponse response = r.path("ws").path("v1").path("timeline")
.path("type_1").path("id_1")
.accept(MediaType.APPLICATION_JSON)
.get(ClientResponse.class);
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
ATSEntity entity = response.getEntity(ATSEntity.class);
TimelineEntity entity = response.getEntity(TimelineEntity.class);
Assert.assertNotNull(entity);
Assert.assertEquals("id_1", entity.getEntityId());
Assert.assertEquals("type_1", entity.getEntityType());
@ -159,12 +159,12 @@ public class TestATSWebServices extends JerseyTest {
@Test
public void testGetEntityFields1() throws Exception {
WebResource r = resource();
ClientResponse response = r.path("ws").path("v1").path("apptimeline")
ClientResponse response = r.path("ws").path("v1").path("timeline")
.path("type_1").path("id_1").queryParam("fields", "events,otherinfo")
.accept(MediaType.APPLICATION_JSON)
.get(ClientResponse.class);
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
ATSEntity entity = response.getEntity(ATSEntity.class);
TimelineEntity entity = response.getEntity(TimelineEntity.class);
Assert.assertNotNull(entity);
Assert.assertEquals("id_1", entity.getEntityId());
Assert.assertEquals("type_1", entity.getEntityType());
@ -177,13 +177,13 @@ public class TestATSWebServices extends JerseyTest {
@Test
public void testGetEntityFields2() throws Exception {
WebResource r = resource();
ClientResponse response = r.path("ws").path("v1").path("apptimeline")
ClientResponse response = r.path("ws").path("v1").path("timeline")
.path("type_1").path("id_1").queryParam("fields", "lasteventonly," +
"primaryfilters,relatedentities")
.accept(MediaType.APPLICATION_JSON)
.get(ClientResponse.class);
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
ATSEntity entity = response.getEntity(ATSEntity.class);
TimelineEntity entity = response.getEntity(TimelineEntity.class);
Assert.assertNotNull(entity);
Assert.assertEquals("id_1", entity.getEntityId());
Assert.assertEquals("type_1", entity.getEntityType());
@ -196,22 +196,22 @@ public class TestATSWebServices extends JerseyTest {
@Test
public void testGetEvents() throws Exception {
WebResource r = resource();
ClientResponse response = r.path("ws").path("v1").path("apptimeline")
ClientResponse response = r.path("ws").path("v1").path("timeline")
.path("type_1").path("events")
.queryParam("entityId", "id_1")
.accept(MediaType.APPLICATION_JSON)
.get(ClientResponse.class);
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
ATSEvents events = response.getEntity(ATSEvents.class);
TimelineEvents events = response.getEntity(TimelineEvents.class);
Assert.assertNotNull(events);
Assert.assertEquals(1, events.getAllEvents().size());
ATSEvents.ATSEventsOfOneEntity partEvents = events.getAllEvents().get(0);
TimelineEvents.EventsOfOneEntity partEvents = events.getAllEvents().get(0);
Assert.assertEquals(2, partEvents.getEvents().size());
ATSEvent event1 = partEvents.getEvents().get(0);
TimelineEvent event1 = partEvents.getEvents().get(0);
Assert.assertEquals(456l, event1.getTimestamp());
Assert.assertEquals("end_event", event1.getEventType());
Assert.assertEquals(1, event1.getEventInfo().size());
ATSEvent event2 = partEvents.getEvents().get(1);
TimelineEvent event2 = partEvents.getEvents().get(1);
Assert.assertEquals(123l, event2.getTimestamp());
Assert.assertEquals("start_event", event2.getEventType());
Assert.assertEquals(0, event2.getEventInfo().size());
@ -219,28 +219,28 @@ public class TestATSWebServices extends JerseyTest {
@Test
public void testPostEntities() throws Exception {
ATSEntities entities = new ATSEntities();
ATSEntity entity = new ATSEntity();
TimelineEntities entities = new TimelineEntities();
TimelineEntity entity = new TimelineEntity();
entity.setEntityId("test id");
entity.setEntityType("test type");
entity.setStartTime(System.currentTimeMillis());
entities.addEntity(entity);
WebResource r = resource();
ClientResponse response = r.path("ws").path("v1").path("apptimeline")
ClientResponse response = r.path("ws").path("v1").path("timeline")
.accept(MediaType.APPLICATION_JSON)
.type(MediaType.APPLICATION_JSON)
.post(ClientResponse.class, entities);
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
ATSPutErrors errors = response.getEntity(ATSPutErrors.class);
Assert.assertNotNull(errors);
Assert.assertEquals(0, errors.getErrors().size());
TimelinePutResponse putResposne = response.getEntity(TimelinePutResponse.class);
Assert.assertNotNull(putResposne);
Assert.assertEquals(0, putResposne.getErrors().size());
// verify the entity exists in the store
response = r.path("ws").path("v1").path("apptimeline")
response = r.path("ws").path("v1").path("timeline")
.path("test type").path("test id")
.accept(MediaType.APPLICATION_JSON)
.get(ClientResponse.class);
assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
entity = response.getEntity(ATSEntity.class);
entity = response.getEntity(TimelineEntity.class);
Assert.assertNotNull(entity);
Assert.assertEquals("test id", entity.getEntityId());
Assert.assertEquals("test type", entity.getEntityType());

View File

@ -224,11 +224,16 @@ public class NodeManager extends CompositeService
new Thread() {
@Override
public void run() {
LOG.info("Notifying ContainerManager to block new container-requests");
containerManager.setBlockNewContainerRequests(true);
LOG.info("Cleaning up running containers on resync");
containerManager.cleanupContainersOnNMResync();
((NodeStatusUpdaterImpl) nodeStatusUpdater ).rebootNodeStatusUpdater();
try {
LOG.info("Notifying ContainerManager to block new container-requests");
containerManager.setBlockNewContainerRequests(true);
LOG.info("Cleaning up running containers on resync");
containerManager.cleanupContainersOnNMResync();
((NodeStatusUpdaterImpl) nodeStatusUpdater).rebootNodeStatusUpdater();
} catch (YarnRuntimeException e) {
LOG.fatal("Error while rebooting NodeStatusUpdater.", e);
shutDown();
}
}
}.start();
}

View File

@ -40,6 +40,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.Dispatcher;
import org.apache.hadoop.yarn.exceptions.NMNotYetReadyException;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
@ -65,6 +66,7 @@ public class TestNodeManagerResync {
private FileContext localFS;
private CyclicBarrier syncBarrier;
private AtomicBoolean assertionFailedInThread = new AtomicBoolean(false);
private AtomicBoolean isNMShutdownCalled = new AtomicBoolean(false);
@Before
public void setup() throws UnsupportedFileSystemException {
@ -137,6 +139,30 @@ public class TestNodeManagerResync {
Assert.assertFalse(assertionFailedInThread.get());
nm.stop();
}
@SuppressWarnings("unchecked")
@Test(timeout=10000)
public void testNMshutdownWhenResyncThrowException() throws IOException,
InterruptedException, YarnException {
NodeManager nm = new TestNodeManager3();
YarnConfiguration conf = createNMConfig();
nm.init(conf);
nm.start();
Assert.assertEquals(1, ((TestNodeManager3) nm).getNMRegistrationCount());
nm.getNMDispatcher().getEventHandler()
.handle(new NodeManagerEvent(NodeManagerEventType.RESYNC));
synchronized (isNMShutdownCalled) {
while (isNMShutdownCalled.get() == false) {
try {
isNMShutdownCalled.wait();
} catch (InterruptedException e) {
}
}
}
Assert.assertTrue("NM shutdown not called.",isNMShutdownCalled.get());
}
private YarnConfiguration createNMConfig() {
YarnConfiguration conf = new YarnConfiguration();
@ -322,4 +348,44 @@ public class TestNodeManagerResync {
}
}
}
class TestNodeManager3 extends NodeManager {
private int registrationCount = 0;
@Override
protected NodeStatusUpdater createNodeStatusUpdater(Context context,
Dispatcher dispatcher, NodeHealthCheckerService healthChecker) {
return new TestNodeStatusUpdaterImpl3(context, dispatcher, healthChecker,
metrics);
}
public int getNMRegistrationCount() {
return registrationCount;
}
@Override
protected void shutDown() {
synchronized (isNMShutdownCalled) {
isNMShutdownCalled.set(true);
isNMShutdownCalled.notify();
}
}
class TestNodeStatusUpdaterImpl3 extends MockNodeStatusUpdater {
public TestNodeStatusUpdaterImpl3(Context context, Dispatcher dispatcher,
NodeHealthCheckerService healthChecker, NodeManagerMetrics metrics) {
super(context, dispatcher, healthChecker, metrics);
}
@Override
protected void registerWithRM() throws YarnException, IOException {
super.registerWithRM();
registrationCount++;
if (registrationCount > 1) {
throw new YarnRuntimeException("Registration with RM failed.");
}
}
}}
}

View File

@ -36,6 +36,7 @@ import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryStore;
import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryWriter;
import org.apache.hadoop.yarn.server.applicationhistoryservice.FileSystemApplicationHistoryStore;
import org.apache.hadoop.yarn.server.applicationhistoryservice.NullApplicationHistoryStore;
import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationAttemptFinishData;
import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationAttemptStartData;
@ -81,8 +82,8 @@ public class RMApplicationHistoryWriter extends CompositeService {
protected synchronized void serviceInit(Configuration conf) throws Exception {
historyServiceEnabled =
conf.getBoolean(YarnConfiguration.YARN_HISTORY_SERVICE_ENABLED,
YarnConfiguration.DEFAULT_YARN_HISTORY_SERVICE_ENABLED);
conf.getBoolean(YarnConfiguration.APPLICATION_HISTORY_ENABLED,
YarnConfiguration.DEFAULT_APPLICATION_HISTORY_ENABLED);
writer = createApplicationHistoryStore(conf);
addIfService(writer);
@ -112,14 +113,15 @@ public class RMApplicationHistoryWriter extends CompositeService {
if (historyServiceEnabled) {
try {
Class<? extends ApplicationHistoryStore> storeClass =
conf.getClass(YarnConfiguration.RM_HISTORY_WRITER_CLASS,
NullApplicationHistoryStore.class, ApplicationHistoryStore.class);
conf.getClass(YarnConfiguration.APPLICATION_HISTORY_STORE,
FileSystemApplicationHistoryStore.class,
ApplicationHistoryStore.class);
return storeClass.newInstance();
} catch (Exception e) {
String msg =
"Could not instantiate ApplicationHistoryWriter: "
+ conf.get(YarnConfiguration.RM_HISTORY_WRITER_CLASS,
NullApplicationHistoryStore.class.getName());
+ conf.get(YarnConfiguration.APPLICATION_HISTORY_STORE,
FileSystemApplicationHistoryStore.class.getName());
LOG.error(msg, e);
throw new YarnRuntimeException(msg, e);
}
@ -214,21 +216,25 @@ public class RMApplicationHistoryWriter extends CompositeService {
@SuppressWarnings("unchecked")
public void applicationStarted(RMApp app) {
dispatcher.getEventHandler().handle(
new WritingApplicationStartEvent(app.getApplicationId(),
ApplicationStartData.newInstance(app.getApplicationId(), app.getName(),
app.getApplicationType(), app.getQueue(), app.getUser(),
app.getSubmitTime(), app.getStartTime())));
if (historyServiceEnabled) {
dispatcher.getEventHandler().handle(
new WritingApplicationStartEvent(app.getApplicationId(),
ApplicationStartData.newInstance(app.getApplicationId(), app.getName(),
app.getApplicationType(), app.getQueue(), app.getUser(),
app.getSubmitTime(), app.getStartTime())));
}
}
@SuppressWarnings("unchecked")
public void applicationFinished(RMApp app, RMAppState finalState) {
dispatcher.getEventHandler().handle(
new WritingApplicationFinishEvent(app.getApplicationId(),
ApplicationFinishData.newInstance(app.getApplicationId(),
app.getFinishTime(), app.getDiagnostics().toString(),
app.getFinalApplicationStatus(),
RMServerUtils.createApplicationState(finalState))));
if (historyServiceEnabled) {
dispatcher.getEventHandler().handle(
new WritingApplicationFinishEvent(app.getApplicationId(),
ApplicationFinishData.newInstance(app.getApplicationId(),
app.getFinishTime(), app.getDiagnostics().toString(),
app.getFinalApplicationStatus(),
RMServerUtils.createApplicationState(finalState))));
}
}
@SuppressWarnings("unchecked")

View File

@ -201,6 +201,22 @@ public class AppSchedulable extends Schedulable {
* Assign a container to this node to facilitate {@code request}. If node does
* not have enough memory, create a reservation. This is called once we are
* sure the particular request should be facilitated by this node.
*
* @param node
* The node to try placing the container on.
* @param priority
* The requested priority for the container.
* @param request
* The ResourceRequest we're trying to satisfy.
* @param type
* The locality of the assignment.
* @param reserved
* Whether there's already a container reserved for this app on the node.
* @return
* If an assignment was made, returns the resources allocated to the
* container. If a reservation was made, returns
* FairScheduler.CONTAINER_RESERVED. If no assignment or reservation was
* made, returns an empty resource.
*/
private Resource assignContainer(FSSchedulerNode node,
ResourceRequest request, NodeType type,
@ -255,17 +271,6 @@ public class AppSchedulable extends Schedulable {
LOG.debug("Node offered to app: " + getName() + " reserved: " + reserved);
}
if (reserved) {
RMContainer rmContainer = node.getReservedContainer();
Priority priority = rmContainer.getReservedPriority();
// Make sure the application still needs requests at this priority
if (app.getTotalRequiredResources(priority) == 0) {
unreserve(priority, node);
return Resources.none();
}
}
Collection<Priority> prioritiesToTry = (reserved) ?
Arrays.asList(node.getReservedContainer().getReservedPriority()) :
app.getPriorities();
@ -338,7 +343,33 @@ public class AppSchedulable extends Schedulable {
return Resources.none();
}
/**
* Called when this application already has an existing reservation on the
* given node. Sees whether we can turn the reservation into an allocation.
* Also checks whether the application needs the reservation anymore, and
* releases it if not.
*
* @param node
* Node that the application has an existing reservation on
*/
public Resource assignReservedContainer(FSSchedulerNode node) {
RMContainer rmContainer = node.getReservedContainer();
Priority priority = rmContainer.getReservedPriority();
// Make sure the application still needs requests at this priority
if (app.getTotalRequiredResources(priority) == 0) {
unreserve(priority, node);
return Resources.none();
}
// Fail early if the reserved container won't fit.
// Note that we have an assumption here that there's only one container size
// per priority.
if (!Resources.fitsIn(node.getReservedContainer().getReservedResource(),
node.getAvailableResource())) {
return Resources.none();
}
return assignContainer(node, true);
}

View File

@ -1046,10 +1046,12 @@ public class FairScheduler extends AbstractYarnScheduler {
reservedAppSchedulable = null;
} else {
// Reservation exists; try to fulfill the reservation
LOG.info("Trying to fulfill reservation for application "
+ reservedAppSchedulable.getApp().getApplicationAttemptId()
+ " on node: " + node);
if (LOG.isDebugEnabled()) {
LOG.debug("Trying to fulfill reservation for application "
+ reservedAppSchedulable.getApp().getApplicationAttemptId()
+ " on node: " + node);
}
node.getReservedAppSchedulable().assignReservedContainer(node);
}
}

View File

@ -77,7 +77,7 @@ public class TestRMApplicationHistoryWriter {
public void setup() {
store = new MemoryApplicationHistoryStore();
Configuration conf = new Configuration();
conf.setBoolean(YarnConfiguration.YARN_HISTORY_SERVICE_ENABLED, true);
conf.setBoolean(YarnConfiguration.APPLICATION_HISTORY_ENABLED, true);
writer = new RMApplicationHistoryWriter() {
@Override