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:
commit
0809f32f4b
|
@ -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.
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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() {}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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];
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
@ -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 =
|
|
@ -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;
|
|
@ -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;
|
||||
}
|
||||
|
|
@ -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
|
||||
*/
|
|
@ -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;
|
||||
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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");
|
||||
|
|
|
@ -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;
|
||||
|
|
@ -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;
|
||||
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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());
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
|
@ -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;
|
|
@ -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));
|
||||
}
|
||||
|
|
@ -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);
|
||||
}
|
||||
|
||||
/**
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -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
|
|
@ -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;
|
||||
}
|
|
@ -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 {
|
||||
}
|
|
@ -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;
|
||||
|
||||
}
|
|
@ -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;
|
|
@ -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");
|
||||
|
|
|
@ -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);
|
|
@ -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();
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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;
|
|
@ -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();
|
||||
}
|
||||
|
|
@ -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;
|
||||
}
|
||||
|
|
@ -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);
|
|
@ -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());
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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.");
|
||||
}
|
||||
}
|
||||
}}
|
||||
}
|
||||
|
|
|
@ -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")
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
Loading…
Reference in New Issue