/
Worker.java
162 lines (142 loc) · 6.47 KB
/
Worker.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
/**
* 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.pulsar.functions.worker;
import com.google.common.util.concurrent.AbstractService;
import java.io.IOException;
import java.net.URI;
import java.util.concurrent.LinkedBlockingQueue;
import lombok.extern.slf4j.Slf4j;
import org.apache.distributedlog.DistributedLogConfiguration;
import org.apache.distributedlog.api.namespace.Namespace;
import org.apache.distributedlog.api.namespace.NamespaceBuilder;
import org.apache.pulsar.client.api.MessageId;
import org.apache.pulsar.client.api.PulsarClient;
import org.apache.pulsar.client.api.PulsarClientException;
import org.apache.pulsar.client.api.Reader;
import org.apache.pulsar.client.api.ReaderConfiguration;
import org.apache.pulsar.functions.runtime.container.FunctionContainerFactory;
import org.apache.pulsar.functions.runtime.container.ThreadFunctionContainerFactory;
import org.apache.pulsar.functions.fs.LimitsConfig;
import org.apache.pulsar.functions.worker.request.ServiceRequestManager;
import org.apache.pulsar.functions.worker.rest.WorkerServer;
@Slf4j
public class Worker extends AbstractService {
private final WorkerConfig workerConfig;
private final LimitsConfig limitsConfig;
private PulsarClient client;
private FunctionRuntimeManager functionRuntimeManager;
private FunctionMetaDataTopicTailer functionMetaDataTopicTailer;
private FunctionContainerFactory functionContainerFactory;
private Thread serverThread;
private Namespace dlogNamespace;
private LinkedBlockingQueue<FunctionAction> actionQueue;
private FunctionActioner functionActioner;
public Worker(WorkerConfig workerConfig, LimitsConfig limitsConfig) {
this.workerConfig = workerConfig;
this.limitsConfig = limitsConfig;
}
@Override
protected void doStart() {
// initialize the dlog namespace
// TODO: move this as part of pulsar cluster initialization later
URI dlogUri;
try {
dlogUri = Utils.initializeDlogNamespace(workerConfig.getZookeeperServers());
} catch (IOException ioe) {
log.error("Failed to initialize dlog namespace at zookeeper {} for storing function packages",
workerConfig.getZookeeperServers(), ioe);
throw new RuntimeException(ioe);
}
// create the dlog namespace for storing function packages
DistributedLogConfiguration dlogConf = Utils.getDlogConf(workerConfig);
try {
this.dlogNamespace = NamespaceBuilder.newBuilder()
.conf(dlogConf)
.clientId("function-worker-" + workerConfig.getWorkerId())
.uri(dlogUri)
.build();
} catch (Exception e) {
log.error("Failed to initialize dlog namespace {} for storing function packages",
dlogUri, e);
throw new RuntimeException(e);
}
// initialize the function metadata manager
try {
log.info("Initialize function metadata manager ...");
this.client = PulsarClient.create(workerConfig.getPulsarServiceUrl());
ServiceRequestManager reqMgr = new ServiceRequestManager(
client.createProducer(workerConfig.getFunctionMetadataTopic()));
this.functionContainerFactory = new ThreadFunctionContainerFactory(limitsConfig.getMaxBufferedTuples(),
workerConfig.getPulsarServiceUrl());
this.actionQueue = new LinkedBlockingQueue<>();
this.functionRuntimeManager = new FunctionRuntimeManager(workerConfig, reqMgr, actionQueue);
this.functionActioner = new FunctionActioner(workerConfig, limitsConfig, functionContainerFactory,
dlogNamespace, actionQueue);
this.functionActioner.start();
ReaderConfiguration readerConf = new ReaderConfiguration();
Reader reader = client.createReader(
workerConfig.getFunctionMetadataTopic(),
MessageId.earliest,
readerConf);
this.functionMetaDataTopicTailer = new FunctionMetaDataTopicTailer(
functionRuntimeManager,
reader);
log.info("Start worker {}...", workerConfig.getWorkerId());
log.info("Worker Configs: {}", workerConfig);
log.info("Limits Configs: {}", limitsConfig);
} catch (Exception e) {
log.error("Failed to create pulsar client to {}",
workerConfig.getPulsarServiceUrl(), e);
throw new RuntimeException(e);
}
WorkerServer server = new WorkerServer(workerConfig, functionRuntimeManager, dlogNamespace);
this.serverThread = new Thread(server, server.getThreadName());
log.info("Start worker server on port {}...", workerConfig.getWorkerPort());
serverThread.start();
log.info("Start worker function state consumer ...");
functionMetaDataTopicTailer.start();
}
@Override
protected void doStop() {
if (null != serverThread) {
serverThread.interrupt();
try {
serverThread.join();
} catch (InterruptedException e) {
log.warn("Worker server thread is interrupted", e);
}
}
if (null != functionMetaDataTopicTailer) {
functionMetaDataTopicTailer.close();
}
if (null != functionRuntimeManager) {
functionRuntimeManager.close();
}
if (null != client) {
try {
client.close();
} catch (PulsarClientException e) {
log.warn("Failed to close pulsar client", e);
}
}
if (null != functionActioner) {
functionActioner.close();
}
}
}