diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/pom.xml hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/pom.xml index f974aee..2aad7bd 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/pom.xml +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/pom.xml @@ -120,6 +120,19 @@ mockito-all test + + + org.apache.phoenix + phoenix-core + 4.3.0 + + + + jline + jline + + + diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java index f1d3d72..d15c3e2 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java @@ -27,12 +27,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.service.CompositeService; -import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineWriteResponse; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineWriterImpl; -import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter; /** * Service that handles writes to the timeline service and writes them to the @@ -46,7 +42,7 @@ public abstract class TimelineCollector extends CompositeService { private static final Log LOG = LogFactory.getLog(TimelineCollector.class); - private TimelineWriter writer; + private TimelineCollectorManager manager; public TimelineCollector(String name) { super(name); @@ -55,11 +51,6 @@ public TimelineCollector(String name) { @Override protected void serviceInit(Configuration conf) throws Exception { super.serviceInit(conf); - writer = ReflectionUtils.newInstance(conf.getClass( - YarnConfiguration.TIMELINE_SERVICE_WRITER_CLASS, - FileSystemTimelineWriterImpl.class, - TimelineWriter.class), conf); - writer.init(conf); } @Override @@ -70,11 +61,10 @@ protected void serviceStart() throws Exception { @Override protected void serviceStop() throws Exception { super.serviceStop(); - writer.stop(); } - public TimelineWriter getWriter() { - return writer; + void attachTo(TimelineCollectorManager m) { + this.manager = m; } /** @@ -99,7 +89,7 @@ public TimelineWriteResponse putEntities(TimelineEntities entities, } TimelineCollectorContext context = getTimelineEntityContext(); - return writer.write(context.getClusterId(), context.getUserId(), + return manager.getWriter().write(context.getClusterId(), context.getUserId(), context.getFlowName(), context.getFlowVersion(), context.getFlowRunId(), context.getAppId(), entities); } diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorManager.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorManager.java index 9a566a2..4008b09 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorManager.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorManager.java @@ -36,6 +36,7 @@ import org.apache.hadoop.http.HttpServer2; import org.apache.hadoop.http.lib.StaticUserWebFilter; import org.apache.hadoop.service.CompositeService; +import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnException; @@ -45,6 +46,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextRequest; import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewCollectorInfoRequest; +import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineWriterImpl; +import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter; import org.apache.hadoop.yarn.webapp.GenericExceptionHandler; import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider; import org.apache.hadoop.yarn.webapp.util.WebAppUtils; @@ -80,6 +83,8 @@ private InetSocketAddress nmCollectorServiceAddress; + private TimelineWriter writer; + static final String COLLECTOR_MANAGER_ATTR_KEY = "collector.manager"; static TimelineCollectorManager getInstance() { @@ -98,7 +103,12 @@ public void serviceInit(Configuration conf) throws Exception { YarnConfiguration.NM_COLLECTOR_SERVICE_ADDRESS, YarnConfiguration.DEFAULT_NM_COLLECTOR_SERVICE_ADDRESS, YarnConfiguration.DEFAULT_NM_COLLECTOR_SERVICE_PORT); - + writer = ReflectionUtils.newInstance( + conf.getClass(YarnConfiguration.TIMELINE_SERVICE_WRITER_CLASS, + FileSystemTimelineWriterImpl.class, TimelineWriter.class), + conf); + writer.init(conf); + super.serviceInit(conf); } @Override @@ -113,6 +123,7 @@ protected void serviceStop() throws Exception { if (timelineRestServer != null) { timelineRestServer.stop(); } + writer.stop(); super.serviceStop(); } @@ -137,6 +148,7 @@ public TimelineCollector putIfAbsent(ApplicationId appId, // cleaned up when the parent shuts down collector.init(getConfig()); collector.start(); + collector.attachTo(this); collectors.put(id, collector); LOG.info("the collector for " + id + " was added"); collectorInTable = collector; @@ -205,6 +217,13 @@ public boolean containsKey(String id) { } /** + * Returns the timeline writer of this manager + */ + TimelineWriter getWriter() { + return writer; + } + + /** * Launch the REST web server for this collector manager */ private void startWebApp() { diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/PhoenixTimelineWriterImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/PhoenixTimelineWriterImpl.java new file mode 100644 index 0000000..6e53528 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/PhoenixTimelineWriterImpl.java @@ -0,0 +1,303 @@ +/** + * 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.timelineservice.storage; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.commons.lang.StringUtils; +import org.apache.commons.logging.Log; +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.service.AbstractService; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineWriteResponse; + +import java.io.IOException; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.Map; +import java.util.Set; + +@Private +@Unstable +public class PhoenixTimelineWriterImpl extends AbstractService + implements TimelineWriter { + + private static final Log LOG = LogFactory.getLog(PhoenixTimelineWriterImpl.class); + private static final String PHOENIX_COL_FAMILY_PLACE_HOLDER = "timeline_cf_placeholder"; + private static final String[] PHOENIX_STORAGE_PK_LIST + = {"cluster", "user", "flow", "run", "appid", "type", "entityid"}; + + /** Default Phoenix JDBC driver name */ + static final String DRIVER_CLASS_NAME = "org.apache.phoenix.jdbc.PhoenixDriver"; + /** Default Phoenix timeline entity table name */ + static final String ENTITY_TABLE_NAME = "TIMELINE_ENTITY"; + /** Default Phoenix timeline config column family */ + static final String CONFIG_COLUMN_FAMILY = "c."; + /** Default Phoenix timeline info column family */ + static final String INFO_COLUMN_FAMILY = "i."; + /** Default separator for Phoenix storage */ + static final String STORAGE_SEPARATOR = ";"; + + /** Connection string to the deployed Phoenix cluster */ + static final String CONN_STRING = "jdbc:phoenix:localhost:2181:/hbase"; + + private Connection conn = null; + + PhoenixTimelineWriterImpl() { + super((PhoenixTimelineWriterImpl.class.getName())); + } + + @Override + protected void serviceInit(Configuration conf) throws Exception { + initConnection(); + super.init(conf); + } + + @Override + protected void serviceStop() throws Exception { + try { + conn.close(); + } catch (SQLException se) { + LOG.error("Failed to close the phoenix connection! " + + se.getLocalizedMessage()); + } + super.serviceStop(); + } + + @Override + public TimelineWriteResponse write(String clusterId, String userId, + String flowName, String flowVersion, long flowRunId, String appId, + TimelineEntities entities) throws IOException{ + TimelineWriteResponse response = new TimelineWriteResponse(); + try { + // Insert entity + String sql = "UPSERT INTO " + ENTITY_TABLE_NAME + + " (" + StringUtils.join(PHOENIX_STORAGE_PK_LIST, ",") + + ", creationtime, modifiedtime, configs) " + + "VALUES (" + StringUtils.repeat("?,", PHOENIX_STORAGE_PK_LIST.length) + + "?, ?, ?)"; + PreparedStatement ps = conn.prepareStatement(sql); + for (TimelineEntity entity : entities.getEntities()) { + int idx = setStringsForPk(ps, clusterId, userId, flowName + flowVersion, + flowRunId, appId, entity, 1); + ps.setLong(idx++, entity.getCreatedTime()); + ps.setLong(idx++, entity.getModifiedTime()); + String configKeys = StringUtils.join( + entity.getConfigs().keySet(), STORAGE_SEPARATOR); + ps.setString(idx++, configKeys); + ps.addBatch(); + } + ps.executeBatch(); + ps.close(); + conn.commit(); + + // Insert configs and info + for (TimelineEntity entity : entities.getEntities()) { + if (entity.getConfigs() == null && entity.getInfo() == null) { + continue; + } + int numPlaceholders = 0; + // Generate the column information for the UPSERT statement + StringBuffer sqlColumns = new StringBuffer( + StringUtils.join(PHOENIX_STORAGE_PK_LIST, ",")); + if (entity.getConfigs() != null) { + appendColumnsSQL(sqlColumns, new ColumnFamilyInfo<>( + CONFIG_COLUMN_FAMILY, entity.getConfigs().keySet())); + numPlaceholders += entity.getConfigs().keySet().size(); + } + if (entity.getInfo() != null) { + appendColumnsSQL(sqlColumns, new ColumnFamilyInfo<>( + INFO_COLUMN_FAMILY, entity.getInfo().keySet())); + numPlaceholders += entity.getInfo().keySet().size(); + } + if (numPlaceholders == 0) { + continue; + } + // Generate the placeholders for the PreparedStatement + StringBuffer placeholders = new StringBuffer(); + placeholders.append( + StringUtils.repeat("?,", PHOENIX_STORAGE_PK_LIST.length)); + // numPlaceholders >= 1 now + placeholders.append("?") + .append(StringUtils.repeat(",?", numPlaceholders - 1)); + // Generate the PreparedStatement + String sqlConfigInfo = new StringBuffer("UPSERT INTO ") + .append(ENTITY_TABLE_NAME).append(" (").append(sqlColumns) + .append(") VALUES(").append(placeholders).append(")").toString(); + PreparedStatement psConfigInfo = conn.prepareStatement(sqlConfigInfo); + // Set up PK info + int idx = setStringsForPk(psConfigInfo, clusterId, userId, + flowName + flowVersion, flowRunId, appId, entity, 1); + // Store config and info + if (entity.getConfigs() != null) { + idx = setStringsForCf(psConfigInfo, entity.getConfigs(), idx); + } + if (entity.getInfo() != null) { + setStringsForCf(psConfigInfo, entity.getInfo(), idx); + } + + // Commit + psConfigInfo.execute(); + psConfigInfo.close(); + conn.commit(); + + } + } catch (SQLException se) { + LOG.error("Failed to add entity to Phoenix " + se.getMessage()); + } + return response; + } + + /** + * Aggregates the entity information to the timeline store based on which + * track this entity is to be rolled up to The tracks along which aggregations + * are to be done are given by {@link TimelineAggregationTrack} + * + * Any errors occurring for individual write request objects will be reported + * in the response. + * + * @param data + * a {@link TimelineEntity} object + * a {@link TimelineAggregationTrack} enum value + * @return a {@link TimelineWriteResponse} object. + * @throws IOException + */ + public TimelineWriteResponse aggregate(TimelineEntity data, + TimelineAggregationTrack track) throws IOException { + return null; + + } + + private void initConnection() { + try { + Class.forName(DRIVER_CLASS_NAME); + conn = DriverManager.getConnection(CONN_STRING); + // Initialize cluster for Phoenix storage + initializeData(); + } catch (SQLException se) { + LOG.error("Failed to connect to phoenix server! " + + se.getLocalizedMessage()); + } catch (ClassNotFoundException e) { + LOG.error("Class not found! " + e.getLocalizedMessage()); + } + } + + private void initializeData() { + try { + // Create tables if necessary + Statement stmt = conn.createStatement(); + // Table schema defined as in YARN-3134 + String sql = "CREATE TABLE IF NOT EXISTS " + ENTITY_TABLE_NAME + + "(cluster VARCHAR NOT NULL, user VARCHAR NOT NULL, " + + "flow VARCHAR NOT NULL, run UNSIGNED_LONG NOT NULL, " + + "appid VARCHAR NOT NULL, type VARCHAR NOT NULL, " + + "entityid VARCHAR NOT NULL, " + + "creationtime UNSIGNED_LONG, modifiedtime UNSIGNED_LONG, " + + "parent VARCHAR, queue VARCHAR, configs VARCHAR, " + + CONFIG_COLUMN_FAMILY + PHOENIX_COL_FAMILY_PLACE_HOLDER + " VARCHAR, " + + INFO_COLUMN_FAMILY + PHOENIX_COL_FAMILY_PLACE_HOLDER + " VARCHAR " + + "CONSTRAINT pk PRIMARY KEY(" + + "cluster, user, flow, run DESC, appid, type, entityid))"; + stmt.executeUpdate(sql); + stmt.close(); + conn.commit(); + } catch (SQLException se) { + LOG.error("Failed in init data " + se.getLocalizedMessage()); + } + return; + } + + private class ColumnFamilyInfo { + String columnFamilyPrefix; + Set columns; + + public ColumnFamilyInfo(String columnFamilyPrefix, Set keyValues) { + this.columnFamilyPrefix = columnFamilyPrefix; + this.columns = keyValues; + } + } + + private StringBuffer appendColumnsSQL( + StringBuffer colNames, ColumnFamilyInfo cfInfo) { + // Prepare the sql template by iterating through all keys + for (K key : cfInfo.columns) { + colNames.append(",").append(cfInfo.columnFamilyPrefix) + .append(key.toString()).append(" VARCHAR"); + } + return colNames; + } + + private int setStringsForCf( + PreparedStatement ps, Map keyValues, int startPos) + throws SQLException{ + int idx = startPos; + for (K key : keyValues.keySet()) { + ps.setString(idx++, keyValues.get(key).toString()); + } + return idx; + } + + private int setStringsForPk(PreparedStatement ps, String clusterId, String userId, + String flowId, Long flowRunId, String appId, + TimelineEntity entity, int startPos) throws SQLException { + int idx = startPos; + ps.setString(idx++, clusterId); + ps.setString(idx++, userId); + ps.setString(idx++, flowId); + ps.setLong(idx++, flowRunId); + ps.setString(idx++, appId); + ps.setString(idx++, entity.getType()); + ps.setString(idx++, entity.getId()); + return idx; + } + + @Private + @VisibleForTesting + ResultSet executeQuery(String sql) { + ResultSet rs = null; + try { + Statement stmt = conn.createStatement(); + rs = stmt.executeQuery(sql); + } catch (SQLException se) { + LOG.error("SQL exception! " + se.getLocalizedMessage()); + } + return rs; + } + + @Private + @VisibleForTesting + void dropEntityTable() { + try { + // Drop table + Statement stmt = conn.createStatement(); + String sql = "DROP TABLE " + ENTITY_TABLE_NAME; + stmt.executeUpdate(sql); + stmt.close(); + } catch (SQLException se) { + LOG.error("Failed in dropping entity table " + se.getLocalizedMessage()); + } + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestPhoenixTimelineWriterImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestPhoenixTimelineWriterImpl.java new file mode 100644 index 0000000..ffa6d60 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestPhoenixTimelineWriterImpl.java @@ -0,0 +1,98 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.timelineservice.storage; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.sql.ResultSet; +import java.sql.SQLException; + +public class TestPhoenixTimelineWriterImpl { + private PhoenixTimelineWriterImpl writer; + + @Before + public void setup() throws Exception { + // TODO: launch a miniphoenix cluster, or else we're directly operating on + // the active Phoenix cluster + YarnConfiguration conf = new YarnConfiguration(); + writer = createPhoenixWriter(conf); + } + + @Test + public void testPhoenixReaderBasic() throws Exception { + // Set up a list of timeline entities and write them back to Phoenix + int numEntity = 12; + TimelineEntities te = + TestTimelineWriterImpl.getStandardTestTimelineEntities(numEntity); + writer.write("cluster_1", "user1", "testFlow", "version1", 1l, "app_test_1", te); + // Verify if we're storing all entities + String sql = "SELECT COUNT(entityid) FROM " + + PhoenixTimelineWriterImpl.ENTITY_TABLE_NAME; + verifySQLWithCount(sql, numEntity, "Number of entities should be "); + // Check config (half of all entities) + sql = "SELECT COUNT(c.config) FROM " + + PhoenixTimelineWriterImpl.ENTITY_TABLE_NAME + "(c.config VARCHAR) "; + verifySQLWithCount(sql, (numEntity / 2), + "Number of entities with config should be "); + // Check info (half of all entities) + sql = "SELECT COUNT(i.info) FROM " + + PhoenixTimelineWriterImpl.ENTITY_TABLE_NAME + "(i.info VARCHAR) "; + verifySQLWithCount(sql, (numEntity / 2), + "Number of entities with info should be "); + // Check config and info (a quarter of all entities) + sql = "SELECT COUNT(entityid) FROM " + + PhoenixTimelineWriterImpl.ENTITY_TABLE_NAME + + "(c.config VARCHAR, i.info VARCHAR) " + + "WHERE c.config IS NOT NULL AND i.info IS NOT NULL"; + verifySQLWithCount(sql, (numEntity / 4), + "Number of entities with both config and info should be "); + } + + @After + public void cleanup() throws Exception { + // Note: it is assumed that we're working on a test only cluster, or else + // this cleanup process will drop the entity table. + writer.dropEntityTable(); + writer.serviceStop(); + } + + private static PhoenixTimelineWriterImpl createPhoenixWriter( + YarnConfiguration conf) throws Exception{ + PhoenixTimelineWriterImpl myWriter = new PhoenixTimelineWriterImpl(); + myWriter.serviceInit(conf); + return myWriter; + } + + private void verifySQLWithCount(String sql, int targetCount, String message) + throws SQLException{ + ResultSet rs = writer.executeQuery(sql); + assertTrue("Result set empty on statement " + sql, rs.next()); + assertNotNull("Fail to execute query " + sql, rs); + assertEquals(message + " " + targetCount, targetCount, rs.getInt(1)); + rs.close(); + } +} diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestTimelineWriterImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestTimelineWriterImpl.java new file mode 100644 index 0000000..1c52959 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestTimelineWriterImpl.java @@ -0,0 +1,53 @@ +/** + * 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.timelineservice.storage; + +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities; +import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; + +public class TestTimelineWriterImpl { + static TimelineEntities getStandardTestTimelineEntities(int listSize) { + TimelineEntities te = new TimelineEntities(); + for (int i = 0; i < listSize; i++) { + TimelineEntity entity = new TimelineEntity(); + String id = "hello" + i; + String type = "world"; + entity.setId(id); + entity.setType(type); + entity.setCreatedTime(1425016501000L + i); + entity.setModifiedTime(1425016502000L + i); + int category = i % 4; + switch (category) { + case 0: + entity.addConfig("config", "config" + i); + // Fall through deliberately + case 1: + entity.addInfo("info", "info text"); + // Fall through deliberately + case 2: + break; + case 3: + entity.addConfig("config", "config" + i); + break; + } + te.addEntity(entity); + } + return te; + } +}