oryx官网
基于oryx-2.1.2
+ oryx
| - app # 基于oryx平台实现的可复用的als、kmeans、rdf算法应用和一个wordcount例子
| - conf # 样例的conf
| - example # wordcount代码
| - oryx-app # als、kmeans、rdf算法应用代码
| - oryx-app-api # 各应用的可定制复用接口
| - oryx-app-common # 各应用的公用代码
| - oryx-app-mllib # als、kmeans、rdf应用的底层算法实现
| - oryx-app-serving # als、kmeans、rdf应用的的servinglayer实现
| - deploy # 与部署运行相关的代码
| - bin # 启动脚本
| - oryx-batch # BatchLayer的二进制main函数
| - oryx-serving # ServingLayer的二进制main函数
| - oryx-speed # SpeedLayer的二进制main函数
| - framework # 框架主要代码实现
| - kafka-util # kafka相关功能
| - oryx-api # 框架的API接口
| - oryx-common # 框架的公用功能
| - oryx-lambda # 框架BatchLayer/SpeedLayer运行、调度、数据分发逻辑代码,这里是框架的主体代码
| - oryx-lambda-serving # 框架ServingLayer的运行逻辑主体代码
| - oryx-ml # 机器学习特别定制的BatchLayer接口,实现了一些机器学习相关的通用逻辑
| - src # 文档等其他文件
为了方便后续的说明,这里举例一个官方wordcount的配置文件例子,文件位于app/conf/wordcount-example.conf
:
# A very basic example config file configuring only the essential elements to
# run the example "word count" application
# Values are examples, appropriate for Cloudera quickstart VM:
kafka-brokers = "quickstart.cloudera:9092"
zk-servers = "quickstart.cloudera:2181"
hdfs-base = "hdfs:///user/cloudera/OryxWordCountExample"
oryx {
id = "WordCountExample"
input-topic {
broker = ${kafka-brokers}
lock = {
master = ${zk-servers}
}
}
update-topic {
broker = ${kafka-brokers}
lock = {
master = ${zk-servers}
}
}
batch {
streaming {
generation-interval-sec = 60
num-executors = 1
executor-cores = 2
executor-memory = "1g"
}
update-class = "com.cloudera.oryx.example.batch.ExampleBatchLayerUpdate"
storage {
data-dir = ${hdfs-base}"/data/"
model-dir = ${hdfs-base}"/model/"
}
ui {
port = 4040
}
}
speed {
streaming {
num-executors = 1
executor-cores = 2
executor-memory = "1g"
}
model-manager-class = "com.cloudera.oryx.example.speed.ExampleSpeedModelManager"
ui {
port = 4041
}
}
serving {
memory = "1000m"
model-manager-class = "com.cloudera.oryx.example.serving.ExampleServingModelManager"
application-resources = "com.cloudera.oryx.example.serving"
api {
port = 8080
}
}
}
完整的配置说明请见:oryx2默认配置文件
usage: oryx-run.sh command [--option value] ...
where command is one of:
batch Run Batch Layer
speed Run Speed Layer
serving Run Serving Layer
kafka-setup Inspect ZK/Kafka config and configure Kafka topics
kafka-tail Follow output from Kafka topics
kafka-input Push data to input topic
and options are one of:
--layer-jar Oryx JAR file, like oryx-{serving,speed,batch}-x.y.z.jar
Defaults to any oryx-*.jar in working dir
--conf Oryx configuration file, like oryx.conf. Defaults to 'oryx.conf'
--app-jar User app JAR file
--jvm-args Extra args to Oryx JVM processes (including drivers and executors)
--deployment Only for Serving Layer now; can be 'yarn' or 'local', Default: local.
--input-file Only for kafka-input. Input file to send
--help Display this messag
这几个目录是对应层的启动主函数,都只是简单的调用了frame的对应层的启动函数而已。
batch
try (BatchLayer,?,?> batchLayer = new BatchLayer<>(ConfigUtils.getDefault())) {
HadoopUtils.closeAtShutdown(batchLayer);
batchLayer.start();
batchLayer.await();
}
try (SpeedLayer,?,?> speedLayer = new SpeedLayer<>(ConfigUtils.getDefault())) {
HadoopUtils.closeAtShutdown(speedLayer);
speedLayer.start();
speedLayer.await();
}
try (ServingLayer servingLayer = new ServingLayer(ConfigUtils.getDefault())) {
JVMUtils.closeAtShutdown(servingLayer);
servingLayer.start();
servingLayer.await();
}
AbstractSparkLayer是batch和speedlayer的基类,因此先介绍AbstractSparkLayer类。
/**
* Encapsulates commonality between Spark-based layer processes,
* {@link com.cloudera.oryx.lambda.batch.BatchLayer} and
* {@link com.cloudera.oryx.lambda.speed.SpeedLayer}
*
* @param input topic key type
* @param input topic message type
*/
public abstract class AbstractSparkLayer<K,M> implements Closeable {
protected AbstractSparkLayer(Config config);
...
protected abstract String getConfigGroup();
protected abstract String getLayerName();
...
protected final JavaStreamingContext buildStreamingContext();
protected final JavaInputDStream> buildInputDStream(JavaStreamingContext streamingContext);
private static void fillInLatestOffsets(Map offsets, Map kafkaParams);
}
protected AbstractSparkLayer(Config config) {
Objects.requireNonNull(config);
log.info("Configuration:\n{}", ConfigUtils.prettyPrint(config));
String group = getConfigGroup();
this.config = config;
String configuredID = ConfigUtils.getOptionalString(config, "oryx.id");
this.id = configuredID == null ? generateRandomID() : configuredID;
this.streamingMaster = config.getString("oryx." + group + ".streaming.master");
this.inputTopic = config.getString("oryx.input-topic.message.topic");
this.inputTopicLockMaster = config.getString("oryx.input-topic.lock.master");
this.inputBroker = config.getString("oryx.input-topic.broker");
this.updateTopic = ConfigUtils.getOptionalString(config, "oryx.update-topic.message.topic");
this.updateTopicLockMaster = ConfigUtils.getOptionalString(config, "oryx.update-topic.lock.master");
// 加载对应的类,整个框架用了很多反射手段
this.keyClass = ClassUtils.loadClass(config.getString("oryx.input-topic.message.key-class"));
this.messageClass = ClassUtils.loadClass(config.getString("oryx.input-topic.message.message-class"));
this.keyDecoderClass = (Class extends Decoder>) ClassUtils.loadClass(config.getString("oryx.input-topic.message.key-decoder-class"), Decoder.class);
this.messageDecoderClass = (Class extends Decoder>) ClassUtils.loadClass(config.getString("oryx.input-topic.message.message-decoder-class"), Decoder.class);
// streaming的计算周期
this.generationIntervalSec = config.getInt("oryx." + group + ".streaming.generation-interval-sec");
// 注意这里,可以添加额外的spark配置,这里会统一读取,并在初始化StreamingContext时设置。
this.extraSparkConfig = new HashMap<>();
for (Map.Entry e : config.getConfig("oryx." + group + ".streaming.config").entrySet()) {
extraSparkConfig.put(e.getKey(), e.getValue().unwrapped());
}
Preconditions.checkArgument(generationIntervalSec > 0);
}
/**
* @return layer-specific config grouping under "oryx", like "batch" or "speed"
*/
protected abstract String getConfigGroup();
/**
* @return display name for layer like "BatchLayer"
*/
protected abstract String getLayerName();
protected final JavaStreamingContext buildStreamingContext() {
log.info("Starting SparkContext with interval {} seconds", generationIntervalSec);
// 初始化sparkconf
SparkConf sparkConf = new SparkConf();
// 下面两部是给测试使用的,正常情况下不会有这种情况发生
// Only for tests, really
if (sparkConf.getOption("spark.master").isEmpty()) {
log.info("Overriding master to {} for tests", streamingMaster);
sparkConf.setMaster(streamingMaster);
}
// Only for tests, really
if (sparkConf.getOption("spark.app.name").isEmpty()) {
String appName = "Oryx" + getLayerName();
if (id != null) {
appName = appName + "-" + id;
}
log.info("Overriding app name to {} for tests", appName);
sparkConf.setAppName(appName);
}
// 设置上面获取的额外的spark设置
for (Map.Entry e : extraSparkConfig.entrySet()) {
sparkConf.setIfMissing(e.getKey(), e.getValue().toString());
}
// 设置一些设置,防止关闭应用时长时间无响应
// Turn this down to prevent long blocking at shutdown
sparkConf.setIfMissing("spark.streaming.gracefulStopTimeout", Long.toString(TimeUnit.MILLISECONDS.convert(generationIntervalSec, TimeUnit.SECONDS)));
sparkConf.setIfMissing("spark.cleaner.ttl", Integer.toString(20 * generationIntervalSec));
// 新建sparkcontext
long generationIntervalMS = TimeUnit.MILLISECONDS.convert(generationIntervalSec, TimeUnit.SECONDS);
JavaSparkContext jsc = JavaSparkContext.fromSparkContext(SparkContext.getOrCreate(sparkConf));
// 新建StreamingContext,注意计算间隔为generationIntervalMS
return new JavaStreamingContext(jsc, new Duration(generationIntervalMS));
}
protected final JavaInputDStream> buildInputDStream(JavaStreamingContext streamingContext) {
// 检查并询问是否创建对应的KafkaTopic
Preconditions.checkArgument(com.cloudera.oryx.kafka.util.KafkaUtils.topicExists(inputTopicLockMaster, inputTopic), "Topic %s does not exist; did you create it?", inputTopic);
if (updateTopic != null && updateTopicLockMaster != null) {
Preconditions.checkArgument(com.cloudera.oryx.kafka.util.KafkaUtils.topicExists(updateTopicLockMaster, updateTopic), "Topic %s does not exist; did you create it?", updateTopic);
}
Map kafkaParams = new HashMap<>();
//kafkaParams.put("zookeeper.connect", inputTopicLockMaster);
String groupID = getGroupID();
kafkaParams.put("group.id", groupID);
// Don't re-consume old messages from input by default
kafkaParams.put("auto.offset.reset", "largest");
kafkaParams.put("metadata.broker.list", inputBroker);
// Newer version of metadata.broker.list:
kafkaParams.put("bootstrap.servers", inputBroker);
// 获取Topic上次消费到的offset,信息保存在inputTopicLockMaster参数配置的zk中
Map offsets = com.cloudera.oryx.kafka.util.KafkaUtils.getOffsets(inputTopicLockMaster, groupID, inputTopic);
fillInLatestOffsets(offsets, kafkaParams);
log.info("Initial offsets: {}", offsets);
// Ugly compiler-pleasing acrobatics:
@SuppressWarnings("unchecked")
Class> streamClass = (Class>) (Class>) MessageAndMetadata.class;
// 创建Kafka的DStream
return KafkaUtils.createDirectStream(streamingContext, keyClass, messageClass, keyDecoderClass, messageDecoderClass, streamClass, kafkaParams, offsets, Functions.>identity());
}
private static void fillInLatestOffsets(Map offsets, Map kafkaParams) {
if (offsets.containsValue(null)) {
Set needOffset = new HashSet<>();
for (Map.Entry entry : offsets.entrySet()) {
if (entry.getValue() == null) {
needOffset.add(entry.getKey());
}
}
log.info("No initial offsets for {}; reading from Kafka", needOffset);
// The high price of calling private Scala stuff:
@SuppressWarnings("unchecked")
scala.collection.immutable.Map kafkaParamsScalaMap = (scala.collection.immutable.Map)scala.collection.immutable.Map$.MODULE$.apply(JavaConversions.mapAsScalaMap(kafkaParams).toSeq());
@SuppressWarnings("unchecked")
scala.collection.immutable.Set needOffsetScalaSet = (scala.collection.immutable.Set)scala.collection.immutable.Set$.MODULE$.apply(JavaConversions.asScalaSet(needOffset).toSeq());
KafkaCluster kc = new KafkaCluster(kafkaParamsScalaMap);
Map leaderOffsets = JavaConversions.mapAsJavaMap(kc.getLatestLeaderOffsets(needOffsetScalaSet).right().get());
for (Map.Entry entry : leaderOffsets.entrySet()) {
TopicAndPartition tAndP = entry.getKey();
// Can't reference LeaderOffset class, so, hack away:
String leaderOffsetString = entry.getValue().toString();
Matcher m = Pattern.compile("LeaderOffset\\([^,]+,[^,]+,([^)]+)\\)").matcher(leaderOffsetString);
Preconditions.checkState(m.matches());
offsets.put(tAndP, Long.valueOf(m.group(1)));
}
}
}
如deploy
解释,SpeedLayer的启动只是调用了对应类的启动命令而已。batch的入口类为com.cloudera.oryx.lambda.speed.SpeedLayer
。
类的定义如下:
/**
* Main entry point for Oryx Speed Layer.
*
* @param type of key read from input topic
* @param type of message read from input topic
* @param type of update message read/written
*/
public final class SpeedLayer<K,M,U> extends AbstractSparkLayer<K,M> {
public SpeedLayer(Config config);
// 类别标示,实现基类接口
protected abstract String getConfigGroup();
protected abstract String getLayerName();
// 任务启停,实现基类接口
public synchronized void start();
public void await();
public synchronized void close();
// 加载对应的类
private SpeedModelManager loadManagerInstance();
private Decoder loadDecoderInstance();
}
public SpeedLayer(Config config)
- 加载SpeedLayer的配置 public SpeedLayer(Config config) {
super(config);
// UpdateTopic的kafka配置
this.updateBroker = config.getString("oryx.update-topic.broker");
this.updateTopic = config.getString("oryx.update-topic.message.topic");
this.maxMessageSize = config.getInt("oryx.update-topic.message.max-size");
this.updateTopicLockMaster = config.getString("oryx.update-topic.lock.master");
// 用户实现的逻辑类
this.modelManagerClassName = config.getString("oryx.speed.model-manager-class");
// update topic的解码类
this.updateDecoderClass = (Class extends Decoder>) ClassUtils.loadClass(config.getString("oryx.update-topic.message.decoder-class"), Decoder.class);
Preconditions.checkArgument(maxMessageSize > 0);
}
@Override
protected String getConfigGroup() {
return "speed";
}
@Override
protected String getLayerName() {
return "SpeedLayer";
}
public synchronized void start();
- 启动streaming任务 public synchronized void start() {
String id = getID();
if (id != null) {
log.info("Starting Speed Layer {}", id);
}
// 初始化StreamingContext,调用基类的函数
streamingContext = buildStreamingContext();
log.info("Creating message stream from topic");
// 创建InputTopic的DStream
JavaInputDStream> dStream = buildInputDStream(streamingContext);
// 映射为KVPair格式
JavaPairDStream pairDStream = dStream.mapToPair(new MMDToTuple2Fn());
// 创建UpdateTopic, 注意这里创建时,从最小的点开始消费
consumer = Consumer.createJavaConsumerConnector(new ConsumerConfig(
ConfigUtils.keyValueToProperties(
"group.id", "OryxGroup-" + getLayerName() + "-" + System.currentTimeMillis(),
"zookeeper.connect", updateTopicLockMaster,
"fetch.message.max.bytes", maxMessageSize,
// Do start from the beginning of the update queue
"auto.offset.reset", "smallest"
)));
// 创建stream并映射为KVPair格式
KafkaStream stream = consumer.createMessageStreams(Collections.singletonMap(updateTopic, 1), new StringDecoder(null), loadDecoderInstance()).get(updateTopic).get(0);
final Iterator> transformed = Iterators.transform(stream.iterator(),
new Function, KeyMessage>() {
@Override
public KeyMessage apply(MessageAndMetadata input) {
return new KeyMessageImpl<>(input.key(), input.message());
}
});
// 加载用户的逻辑类
modelManager = loadManagerInstance();
// 创建UpdateTopic的消费线程,注意这里没有使用DStream消费UpdateTopic,而是使用了一个单独线程进行消费
new Thread(new LoggingRunnable() {
@Override
public void doRun() {
try {
// 调用consume函数消费stream
modelManager.consume(transformed, streamingContext.sparkContext().hadoopConfiguration());
} catch (Throwable t) {
log.error("Error while consuming updates", t);
close();
}
}
}, "OryxSpeedLayerUpdateConsumerThread").start();
// 对于InputTopic,进行RDD的正常变换,对于InputTopic可以使用DStream的API。这里用到了`SpeedLayerUpdate`函数类。
pairDStream.foreachRDD(new SpeedLayerUpdate<>(modelManager, updateBroker, updateTopic));
// 记录消费进度,这里用到了`UpdateOffsetsFn`函数类。
dStream.foreachRDD(new UpdateOffsetsFn(getGroupID(), getInputTopicLockMaster()));
log.info("Starting Spark Streaming");
// 启动StreamingContext
streamingContext.start();
}
//////////////////////////////////////////
// `SpeedLayerUpdate`函数类 - 用作在SpeedLayer和用户函数之间加了一层调用函数。
//////////////////////////////////////////
/**
* Main Spark Streaming function for the speed layer that collects and publishes update to
* a Kafka topic.
*
* @param type of key read from input topic
* @param type of message read from input topic
* @param type of update message read/written
*/
public final class SpeedLayerUpdate<K,M,U> implements Function<JavaPairRDD<K,M>,Void> {
private static final Logger log = LoggerFactory.getLogger(SpeedLayerUpdate.class);
private final SpeedModelManager modelManager;
private final String updateBroker;
private final String updateTopic;
public SpeedLayerUpdate(SpeedModelManager modelManager,
String updateBroker,
String updateTopic) {
this.modelManager = modelManager;
this.updateBroker = updateBroker;
this.updateTopic = updateTopic;
}
// 主要行为 - 处理InputTopic的每个RDD
@Override
public Void call(JavaPairRDD newData) throws IOException {
if (newData.isEmpty()) {
log.debug("RDD was empty");
} else {
// 调用用户的RDD处理逻辑
Iterable updates = modelManager.buildUpdates(newData);
// 分发输出,默认写到UpdateTopic中
if (updates != null) {
try (TopicProducer producer = new TopicProducerImpl<>(updateBroker, updateTopic, true)) {
// 注意这里,写入producer的时候,带着"UP"标签。
for (U update : updates) { producer.send("UP", update);}
}
}
}
return null;
}
}
//////////////////////////////////////////
// `UpdateOffsetsFn`函数类 - 用作更新zk中的消费OFFSET。
//////////////////////////////////////////
/**
* Function that reads offset range from latest RDD in a streaming job, and updates
* Zookeeper/Kafka with the latest offset consumed.
*
* @param RDD element's key type (not used)
* @param RDD element's value type (not used)
*/
public final class UpdateOffsetsFn<K,M> implements Function<JavaRDD<MessageAndMetadata<K,M>>,Void> {
private static final Logger log = LoggerFactory.getLogger(UpdateOffsetsFn.class);
private final String group;
private final String inputTopicLockMaster;
public UpdateOffsetsFn(String group, String inputTopicLockMaster) {
this.group = group;
this.inputTopicLockMaster = inputTopicLockMaster;
}
/**
* @param javaRDD RDD whose underlying RDD must be an instance of {@link HasOffsetRanges},
* such as {@code KafkaRDD}
* @return null
*/
@Override
public Void call(JavaRDD> javaRDD) {
// 读取每个RDD中的OFFSET记录
OffsetRange[] ranges = ((HasOffsetRanges)javaRDD.rdd()).offsetRanges();
Map newOffsets = new HashMap<>(ranges.length);
for (OffsetRange range : ranges) {
// 组装KAFKA需要的OFFSET数据格式
newOffsets.put(new TopicAndPartition(range.topic(), range.partition()), range.untilOffset());
}
log.info("Updating offsets: {}", newOffsets);
// 更新ZK中的OFFSET记录
KafkaUtils.setOffsets(inputTopicLockMaster, group, newOffsets);
return null;
}
}
注意start函数是synchronized
,线程互斥。
这个函数是SteamingLayer的主要逻辑,可以看到InputTopic的消费走的是正常的sparkapi,但UpdateTopic的消费走的是单独线程。
这样的话用户就需要考虑到两个线程之间的加锁问题。同时单线程消费大量数据可能会有性能问题。
public void await()
- 等待StreamingContext程序结束 public void await() {
Preconditions.checkState(streamingContext != null);
log.info("Spark Streaming is running");
// 等待结束
streamingContext.awaitTermination();
}
public synchronized void close()
- 释放资源 public synchronized void close() {
// 调用用户的close函数
if (modelManager != null) {
log.info("Shutting down model manager");
modelManager.close();
modelManager = null;
}
// 关闭UpdateTopic
if (consumer != null) {
log.info("Shutting down consumer");
consumer.commitOffsets(); // 注意这里,记录了自己的消费offset
consumer.shutdown();
consumer = null;
}
// 关闭StreamingContext
if (streamingContext != null) {
log.info("Shutting down Spark Streaming; this may take some time");
streamingContext.stop(true, true);
streamingContext = null;
}
}
注意close函数也是synchronized
。
2.1.3. 总结
1. SpeedLayer将InputTopic当作正常的DStream使用,框架封装了Offset的更新等逻辑。
2. UpdateTopic启动了一个单独线程,直接使用kafka的api完成,因此不需要更新zk的Offset记录。
3. BuildUpdate的输出默认输出到UpdateTopic中,并带有”UP”标签。
如deploy
解释,batchlayer的启动只是调用了对应类的启动命令而已。batch的入口类为com.cloudera.oryx.lambda.batch.BatchLayer
。
/**
* Main entry point for Oryx Batch Layer.
*
* @param type of key read from input topic
* @param type of message read from input topic
* @param type of model message written
*/
public final class BatchLayer<K,M,U> extends AbstractSparkLayer<K,M> {
public BatchLayer(Config config);
// 类别标示,实现基类接口
protected abstract String getConfigGroup();
protected abstract String getLayerName();
// 启动停止任务
public synchronized void start();
public void await();
public synchronized void close();
// 加载Update类
private BatchLayerUpdate loadUpdateInstance();
}
public BatchLayer(Config config);
- 加载重要参数 public BatchLayer(Config config) {
super(config);
// 数据存储使用的类
this.keyWritableClass = ClassUtils.loadClass(config.getString("oryx.batch.storage.key-writable-class"), Writable.class);
this.messageWritableClass = ClassUtils.loadClass(config.getString("oryx.batch.storage.message-writable-class"), Writable.class);
// Update使用的用户类
this.updateClassName = config.getString("oryx.batch.update-class");
// 数据存储位置
this.dataDirString = config.getString("oryx.batch.storage.data-dir");
this.modelDirString = config.getString("oryx.batch.storage.model-dir");
// 数据存储时间
this.maxDataAgeHours = config.getInt("oryx.batch.storage.max-age-data-hours");
Preconditions.checkArgument(!dataDirString.isEmpty());
Preconditions.checkArgument(!modelDirString.isEmpty());
Preconditions.checkArgument(maxDataAgeHours >= 0 || maxDataAgeHours == NO_MAX_DATA_AGE);
}
@Override
protected String getConfigGroup() {
return "batch";
}
@Override
protected String getLayerName() {
return "BatchLayer";
}
public synchronized void start();
- 启动任务 public synchronized void start() {
String id = getID();
if (id != null) {
log.info("Starting Batch Layer {}", id);
}
// 创建StreamingContext
streamingContext = buildStreamingContext();
// 开启检查点!这个很重要!
Path checkpointPath = new Path(new Path(modelDirString), ".checkpoint");
log.info("Setting checkpoint dir to {}", checkpointPath);
streamingContext.sparkContext().setCheckpointDir(checkpointPath.toString());
// 创建InputDStream
log.info("Creating message stream from topic");
JavaInputDStream> dStream = buildInputDStream(streamingContext);
JavaPairDStream pairDStream = dStream.mapToPair(new MMDToTuple2Fn());
Class keyClass = getKeyClass();
Class messageClass = getMessageClass();
// 使用`BatchUpdateFunction`函数类处理数据,内部逻辑是调用用户的具体实现类来进行。
pairDStream.foreachRDD(new BatchUpdateFunction<>(getConfig(), keyClass, messageClass, keyWritableClass, messageWritableClass, dataDirString, modelDirString, loadUpdateInstance(), streamingContext));
// 将接收到的数据存入HDFS,数据保存在dataDirString下
// "Inline" saveAsNewAPIHadoopFiles to be able to skip saving empty RDDs
pairDStream.foreachRDD(new SaveToHDFSFunction<>(dataDirString + "/oryx", "data", keyClass, messageClass, keyWritableClass, messageWritableClass, streamingContext.sparkContext().hadoopConfiguration()));
// 更新zk中的offset
dStream.foreachRDD(new UpdateOffsetsFn(getGroupID(), getInputTopicLockMaster()));
// 删除HDFS上旧的数据文件,数据保存在dataDirString下
if (maxDataAgeHours != NO_MAX_DATA_AGE) {
dStream.foreachRDD(new DeleteOldDataFn>(streamingContext.sparkContext().hadoopConfiguration(), dataDirString, maxDataAgeHours));
}
log.info("Starting Spark Streaming");
// 启动streaming处理
streamingContext.start();
}
可以看到具体逻辑是调用用户的处理逻辑,同时保存一份数据到HDFS下,供后续迭代当作窗口数据使用。
BatchUpdateFunction
- 用户处理逻辑的封装类 /**
* Framework for executing the batch layer update, and storing data to persistent storage,
* in the context of a streaming framework.
*
* @param type of key read from input topic
* @param type of message read from input topic
* @param type of model message written
*/
final class BatchUpdateFunction implements Function2,Time,Void> {
...
// 主要调用逻辑
@Override
public Void call(JavaPairRDD newData, Time timestamp) throws IOException, InterruptedException {
if (newData.isEmpty()) {
log.info("No data in current generation's RDD; nothing to do");
return null;
}
log.info("Beginning update at {}", timestamp);
Configuration hadoopConf = sparkContext.hadoopConfiguration();
if (hadoopConf.getResource("core-site.xml") == null) {
log.warn("Hadoop config like core-site.xml was not found; " + "is the Hadoop config directory on the classpath?");
}
// 读取窗口数据,创建窗口RDD
JavaPairRDD pastData;
Path inputPathPattern = new Path(dataDirString + "/*/part-*");
FileSystem fs = FileSystem.get(inputPathPattern.toUri(), hadoopConf);
FileStatus[] inputPathStatuses = fs.globStatus(inputPathPattern);
if (inputPathStatuses == null || inputPathStatuses.length == 0) {
log.info("No past data at path(s) {}", inputPathPattern);
pastData = null;
} else {
log.info("Found past data at path(s) like {}", inputPathStatuses[0].getPath());
Configuration updatedConf = new Configuration(hadoopConf);
updatedConf.set(FileInputFormat.INPUT_DIR, joinFSPaths(fs, inputPathStatuses));
// 读取窗口数据
@SuppressWarnings("unchecked")
JavaPairRDD pastWritableData = (JavaPairRDD)sparkContext.newAPIHadoopRDD(updatedConf, SequenceFileInputFormat.class, keyWritableClass, messageWritableClass);
// 转换为KVPair格式
pastData = pastWritableData.mapToPair(new WritableToValueFunction<>(keyClass, messageClass, keyWritableClass, messageWritableClass));
}
// 执行用户逻辑,这里由用户决定推送哪些给producer
if (updateTopic == null || updateBroker == null) {
log.info("Not producing updates to update topic since none was configured");
updateInstance.runUpdate(sparkContext, timestamp.milliseconds(), newData, pastData, modelDirString, null);
} else {
// This TopicProducer should not be async; sends one big model generally and
// needs to occur before other updates reliably rather than be buffered
try (TopicProducer producer = new TopicProducerImpl<>(updateBroker, updateTopic, false)) {
updateInstance.runUpdate(sparkContext, timestamp.milliseconds(), newData, pastData, modelDirString, producer);
}
}
return null;
}
...
}
SaveToHDFSFunction
- 保存到HDFS功能 /**
* Function that saves RDDs to HDFS -- only if they're non empty, to prevent creation
* of many small empty files if data is infrequent but the model interval is short.
*/
final class SaveToHDFSFunction implements Function2,Time,Void> {
@Override
public Void call(JavaPairRDD rdd, Time time) {
if (rdd.isEmpty()) {
log.info("RDD was empty, not saving to HDFS");
} else {
// 保存文件到`prefix-{timestemp}.suffix/`目录下
String file = prefix + "-" + time.milliseconds() + "." + suffix;
log.info("Saving RDD to HDFS at {}", file);
// 转换KVPair到Writable对象,并保存到HDFS中
rdd.mapToPair(new ValueToWritableFunction<>(keyClass, messageClass, keyWritableClass, messageWritableClass)
).saveAsNewAPIHadoopFile(file, keyWritableClass, messageWritableClass, SequenceFileOutputFormat.class, hadoopConf);
}
return null;
}
}
DeleteOldDataFn
- 删除过期数据 /**
* Function that deletes old data, if applicable, at each batch interval.
*
* @param unused
*/
public final class DeleteOldDataFn<T> implements Function<JavaRDD<T>,Void> {
@Override
public Void call(JavaRDD ignored) throws IOException {
// 构造路径,参照上面的SaveToHDFSFunction
Path dataDirPath = new Path(dataDirString + "/*");
FileSystem fs = FileSystem.get(dataDirPath.toUri(), hadoopConf);
FileStatus[] inputPathStatuses = fs.globStatus(dataDirPath);
if (inputPathStatuses != null) {
long oldestTimeAllowed = System.currentTimeMillis() - TimeUnit.MILLISECONDS.convert(maxDataAgeHours, TimeUnit.HOURS);
for (FileStatus status : inputPathStatuses) {
if (status.isDirectory()) {
Path subdir = status.getPath();
Matcher m = DATA_SUBDIR_PATTERN.matcher(subdir.getName());
// 查看目录时间是否过期,并删除过期目录
if (m.find() && Long.parseLong(m.group(1)) < oldestTimeAllowed) {
log.info("Deleting old data at {}", subdir);
try {
fs.delete(subdir, true);
} catch (IOException e) {
log.warn("Unable to delete {}; continuing", subdir, e);
}
}
}
}
}
return null;
}
}
ValueToWritableFunction
, WritableToValueFunction
, ValueWritableConverter
这几个类用作类型转换。TopicProducerImpl
- Kafka的对接类 /**
* Wraps access to a Kafka message topic {@link Producer}.
*
* @param key type to send
* @param message type to send
*/
public final class TopicProducerImpl<K,M> implements TopicProducer<K,M> {
private final String updateBroker;
private final String topic;
private final boolean async;
private Producer producer;
public TopicProducerImpl(String updateBroker, String topic, boolean async) {
this.updateBroker = updateBroker;
this.topic = topic;
this.async = async;
}
@Override
public String getUpdateBroker() {
return updateBroker;
}
@Override
public String getTopic() {
return topic;
}
// 生成一个新的KafkaProducer。注意其中的`"producer.type", async ? "async" : "sync",`决定了是否是异步。
private synchronized Producer getProducer() {
// Lazy init; also handles case where object has been serialized and Producer
// needs to be recreated
if (producer == null) {
producer = new Producer<>(new ProducerConfig(ConfigUtils.keyValueToProperties(
"metadata.broker.list", updateBroker,
"serializer.class", StringEncoder.class.getName(),
"producer.type", async ? "async" : "sync",
"queue.buffering.max.ms", 1000, // Make configurable?
"batch.num.messages", 100,
"compression.codec", "gzip",
"compressed.topics", topic
)));
}
return producer;
}
@Override
public void send(K key, M message) {
getProducer().send(new KeyedMessage<>(topic, key, message));
}
@Override
public synchronized void close() {
if (producer != null) {
producer.close();
}
}
}
简单来说,用户继承AbstractServingModelManager实现自己的类,ServingLayer会启动Tomcat,同时启动一个消费Kafka的UpdateTopic线程,执行用户的处理类的consume函数。在请求获取model的时候,会执行用户类的getModel函数获取。
待补充
/**
* Also counts and emits counts of number of distinct words that occur with words.
* Listens for updates from the Batch Layer, which give the current correct count at its
* last run. Updates these counts approximately in response to the same data stream
* that the Batch Layer sees, but assumes all words seen are new and distinct, which is only
* approximately true. Emits updates of the form "word,count".
*/
public final class ExampleSpeedModelManager implements SpeedModelManager<String,String,String> {
private final Map distinctOtherWords = Collections.synchronizedMap(new HashMap());
@Override
// 消费UpdateTopic的数据,并更新自己的数据Base
public void consume(Iterator> updateIterator, Configuration hadoopConf) throws IOException {
while (updateIterator.hasNext()) {
KeyMessage km = updateIterator.next();
String key = km.getKey();
String message = km.getMessage();
switch (key) {
case "MODEL":
@SuppressWarnings("unchecked")
Map model = (Map) new ObjectMapper().readValue(message, Map.class);
distinctOtherWords.keySet().retainAll(model.keySet());
for (Map.Entry entry : model.entrySet()) {
distinctOtherWords.put(entry.getKey(), entry.getValue());
}
break;
case "UP":
// ignore
break;
default:
throw new IllegalArgumentException("Unknown key " + key);
}
}
}
@Override
// 流式计算新数据
public Iterable buildUpdates(JavaPairRDD newData) {
List updates = new ArrayList<>();
for (Map.Entry entry : ExampleBatchLayerUpdate.countDistinctOtherWords(newData).entrySet()) {
String word = entry.getKey();
int count = entry.getValue();
int newCount;
synchronized (distinctOtherWords) {
Integer oldCount = distinctOtherWords.get(word);
newCount = oldCount == null ? count : oldCount + count;
distinctOtherWords.put(word, newCount);
}
updates.add(word + "," + newCount);
}
// 返回的是发送到UpdateTopic的数据集合
return updates;
}
@Override
public void close() {
// do nothing
}
}
/**
* Input keys are ignored. Values are treated as lines of space-separated text. The job
* counts, for each word, the number of distinct other words that co-occur in some line
* of text in the input. These are written as a "MODEL" update, where the word-count mapping
* is written as a JSON string.
*/
public final class ExampleBatchLayerUpdate implements BatchLayerUpdate<String,String,String> {
@Override
// 创建Batch数据更新
public void runUpdate(JavaSparkContext sparkContext,
long timestamp,
JavaPairRDD newData,
JavaPairRDD pastData,
String modelDirString,
TopicProducer modelUpdateTopic) throws IOException {
JavaPairRDD allData = pastData == null ? newData : newData.union(pastData);
String modelString;
try {
modelString = new ObjectMapper().writeValueAsString(countDistinctOtherWords(allData));
} catch (JsonProcessingException jpe) {
throw new IOException(jpe);
}
// 发送到UpdateTopic,注意MODEL对应SpeedLayer的consume函数
modelUpdateTopic.send("MODEL", modelString);
}
// 具体逻辑,使用DStream的API
public static Map countDistinctOtherWords(JavaPairRDD data) {
return data.values().flatMapToPair(new PairFlatMapFunction() {
@Override
public Iterable> call(String line) {
List> result = new ArrayList<>();
Set distinctTokens = new HashSet<>(Arrays.asList(line.split(" ")));
for (String a : distinctTokens) {
for (String b : distinctTokens) {
if (!a.equals(b)) {
result.add(new Tuple2<>(a, b));
}
}
}
return result;
}
}).distinct().groupByKey().mapValues(new Function,Integer>() {
@Override
public Integer call(Iterable values) {
int count = 0;
for (String v : values) {
count++;
}
return count;
}
}).collectAsMap();
}
}
/**
* Reads models and updates produced by the Batch Layer and Speed Layer. Models are maps, encoded as JSON
* strings, mapping words to count of distinct other words that appear with that word in an input line.
* Updates are "word,count" pairs representing new counts for a word. This class manages and exposes the
* mapping to the Serving Layer applications.
*/
public final class ExampleServingModelManager extends AbstractServingModelManager<String> {
private final Map distinctOtherWords = Collections.synchronizedMap(new HashMap());
public ExampleServingModelManager(Config config) {
super(config);
}
@Override
// 从UpdateTopic中消费Batch和Speed产出的数据,并更新Model
public void consume(Iterator> updateIterator, Configuration hadoopConf) throws IOException {
while (updateIterator.hasNext()) {
KeyMessage km = updateIterator.next();
String key = km.getKey();
String message = km.getMessage();
switch (key) {
case "MODEL":
@SuppressWarnings("unchecked")
Map model = (Map) new ObjectMapper().readValue(message, Map.class);
distinctOtherWords.keySet().retainAll(model.keySet());
for (Map.Entry entry : model.entrySet()) {
distinctOtherWords.put(entry.getKey(), entry.getValue());
}
break;
case "UP":
String[] wordCount = message.split(",");
distinctOtherWords.put(wordCount[0], Integer.valueOf(wordCount[1]));
break;
default:
throw new IllegalArgumentException("Unknown key " + key);
}
}
}
@Override
// 获取Model数据
public ServingModel getModel() {
return new ExampleServingModel(distinctOtherWords);
}
}