簡介:本文主要講解Apache NIFI的調度策略,對象主要是針對Processor組件。本文假定讀者已經對Apache NIFI有了一定的瞭解和使用經驗,同時作者也儘可能的去講解的更透徹,使得本文儘可能讓對NIFI接觸不深的讀者也能夠看懂。
NIFI的調度策略
打開任意一個Processsor,在其配置頁面SCHEDULING頁籤我們可以配置它的調度策略,如下圖所示:
在流程中有一類的Processor的實例是不允許傳入FlowFIle的,我們姑且可以稱之爲流程源結點
(第一個節點)。這類Processor實例的調度策略只有兩種,而其他的Processor實例的調度策略有三種。(注意,這裏強調的是Processor實例,有些Processor在源組件
位置時是兩種調度策略,沒有Event策略,而當這些Processor不處於源組件
位置時,它會有三種調度策略)
-
Timer driven: 這是調度策略的默認模式。處理器會定期運行。處理器運行的時間間隔由
Run schedule
選項定義。 -
Event driven: 如果選擇此模式,則由event觸發處理器運行,並且該event在FlowFiles進入到該處理器的上游Connection時發生。該模式當前被認爲是實驗性的,
並非所有處理器都支持
(Processor類使用@EventDriven註解標註的組件是支持Event driven的)。選擇此模式後,Run schedule
選項不可配置,因爲不會觸發處理器定期
運行,而是由event觸發。此外,這是可以將Concurrent tasks
選項設置爲0的唯一模式。在這種情況下,線程數僅受Event-Driven Thread Pool
的大小限制。 -
CRON driven: 當使用CRON驅動的調度模式時,處理器被調度爲定期運行(比如每天凌晨調度運行),
類似於計時器驅動的調度模式
,CRON驅動模式以增加配置複雜性爲代價提供了更大的靈活性。 CRON驅動的調度值是由六個必填字段和一個可選字段組成的字符串,每個字段之間用空格分隔。
好了,以上就是本文全部內容。
全劇終。。。
好吧,上面所述都是理論知識,在官方文檔裏都能看到,下面我們進一步對這些調度策略進行探索總結。
Timer driven
Timer driven
是我們最常用的調度策略了,簡單易懂,10 sec
就是每隔10秒調度一次。
可識別的後綴如下所示:
- 納秒:“ns”, “nano”, “nanos”, “nanosecond”, “nanoseconds”
- 毫秒:“ms”, “milli”, “millis”, “millisecond”, “milliseconds”
- 秒:“s”, “sec”, “secs”, “second”, “seconds”
- 分鐘:“m”, “min”, “mins”, “minute”, “minutes”
- 小時:“h”, “hr”, “hrs”, “hour”, “hours”
- 天:“d”, “day”, “days”
- 周:“w”, “wk”, “wks”, “week”, “weeks”
疑問1
那麼第一個問題來了,比如說每隔10秒調度一次,是什麼意思?是從0秒開始,10秒,20秒,30秒。。。這樣的每次去執行調度嘛?還是每次任務結束後開始計時?
下面我們來實際求證一下。
新拉取一個ExecuteGroovyScript
組件,選擇Timer driven
並設置2秒運行一次,然後在Script Body
配置中添加Groovy代碼
//創建一個流文件
flowFile = session.create()
//添加一個屬性,在FlowFIle中記錄一個時間,姑且把這個時間當做本次調度開始時間
flowFile = session.putAttribute(flowFile, 'Time', String.valueOf(System.currentTimeMillis()))
//然後休眠3秒
try {
Thread.sleep(3000);
} catch (InterruptedException e) {
e.printStackTrace();
}
//將FlowFIle傳輸到success
session.transfer(flowFile, REL_SUCCESS)
點擊運行後生成了三個流文件
我們分別來看一下這3個流文件的Time
屬性
1591270393098
+5141 大約是5秒
1591270398239
+5012 大約是5秒
1591270403251
所以,結果顯而易見了,這裏是每次任務結束後開始計時(後面疑問5裏會提到源碼,源碼裏就是上一次任務結束後計時,時間一過,然後開始下次任務)。
疑問2
第二個問題 如果我們安排處理器每0秒運行一次(默認情況下),會發生什麼?它會一直運行,消耗大量資源嗎?
答案顯然是不會的!(如果這點都做不好,還搞啥子Apache頂級項目嘛)
在NIFI安裝目錄conf下的nifi.properties中有如下配置,隊列中沒有數據的時候也就是Processor沒有可處理的數據,那麼我們在這裏配置隔多久再去調度檢查一次組件是否有可做的有工作。
# If a component has no work to do (is "bored"), how long should we wait before checking again for work?
nifi.bored.yield.duration=10 millis
假如我們使用的是默認配置,那麼意思是說雖然我們配置了處理器每0秒運行一次,但當Processor沒有工作要做時(可以簡單理解爲上游Connection是空的),它會等10 millis
然後再調度一次檢查組件是否有工作要做(在後面的疑問5裏會有源碼說明到這個10ms)。
疑問3
看到這裏使用過Apache NIFI的人可能會有疑問了,怎麼會這樣,我們在運行流程的時候,比如下圖UpdateAttribute
設置的每0秒運行一次,它的上游Connection是空的,我們觀察它並沒有被調度啊?(組件方塊右上角根本沒有顯示任何數字)
然後我們要明確一點,Processor右上角的那個數字的含義是Active Tasks
Active Tasks
:該處理器當前正在執行的任務數(有幾個任務在調用Processor的onTrigger方法)。此數字受Processor配置對話框的Scheduling
選項卡中的Concurrent tasks
設置約束。在這裏,我們可以看到處理器當前正在執行一項任務。如果NiFi實例是集羣的,則此值表示集羣中所有節點上當前正在執行的任務數。
額外說一些,那麼顯示出來的這個Acrive Task
是怎麼來的呢?首先在Apache NIFI中有一個對象StandardProcessorNode
(可以簡單這麼理解,我們對於一個組件的調度、併發等等配置以及對這個組件的監控、跟蹤等等信息都是存儲在這個對象裏,每個Processor實例都會對應一個StandardProcessorNode實例),ProcessorNode是線程安全的。在StandardProcessorNode
中有這麼這麼幾段代碼:
@Override
public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) {
//獲取ProcessorNode所指向的Processor
final Processor processor = processorRef.get().getProcessor();
//Active Task +1
activateThread();
try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(getExtensionManager(), processor.getClass(), processor.getIdentifier())) {
//這裏調用的是Processor的onTrigger方法邏輯
processor.onTrigger(context, sessionFactory);
} finally {
//Active Task -1
deactivateThread();
}
}
而activateThread()
方法
private final Map<Thread, ActiveTask> activeThreads = new HashMap<>(48);
private synchronized void activateThread() {
final Thread thread = Thread.currentThread();
final Long timestamp = System.currentTimeMillis();
activeThreads.put(thread, new ActiveTask(timestamp));
}
而在NIFI Web Api裏,在生成一個Processor的狀態信息時會調用public synchronized List<ActiveThreadInfo> getActiveThreads()
方法,進而就能夠獲取到Acrive Task
數值,這裏就不展開源碼說了,感興趣的讀者直接順着上面說的方法查看調用就可以找到了。
那我們來手動證明一次,當Processor沒有需要處理的數據時,不會觸發Processor的onTrigger方法,我們複用上面的groovy組件,加一行打印日誌的代碼
// 打印警告日誌
log.warn('我被調度了')
//創建一個流文件
flowFile = session.create()
//添加一個屬性,在FlowFIle中記錄一個時間,姑且把這個時間當做本次調度開始時間
flowFile = session.putAttribute(flowFile, 'Time', String.valueOf(System.currentTimeMillis()))
//然後休眠3秒
try {
Thread.sleep(3000);
} catch (InterruptedException e) {
e.printStackTrace();
}
//將FlowFIle傳輸到success
session.transfer(flowFile, REL_SUCCESS)
然後在ExecuteGroovyScript
上游添加一個組件GenerateFlowFile
用於生成流文件。這是持續一段時間後的截圖,我們觀察組件右上角並沒有日誌輸出表明這個組件的onTrigger並沒有被調用
作爲對比,我們發送一個流文件,就能觀察到日誌輸出:
總結一下:我們配置了處理器每0秒運行一次,但當Processor沒有工作要做時,它會等10 millis
然後再檢查一次是否有工作要做,是不會觸發Processor運行任務的(不會調Processor的onTrigger方法)。
這裏我們說的比較清楚,Processor沒有工作要做導致了沒有Active Task(不會觸發Processor的onTrigger方法),而不是說沒有線程運行或者沒有調度發生。
疑問4
那麼怎麼判斷Processor是否有工作要做?
首先我們看到,一個叫ConnectableTask
的實例會去調用StandardProcessorNode
的onTrigger
方法,執行的地方叫invoke()
而調用ConnectableTask
的invoke()
方法的有兩個agent:QuartzSchedulingAgent
對應CRON driven TimerDrivenSchedulingAgent
對應Timer driven。先不管agent,在invoke()
方法會調用isWorkToDo()
來判斷這個組件實例是否有工作要做。
private boolean isWorkToDo() {
//Connectables意思是可以連接的組件,這裏判斷當前的connectable的所有上游的Connection是否都是來自於它自己
boolean hasNonLoopConnection = Connectables.hasNonLoopConnection(connectable);
//對FUNNEL組件特殊處理 這裏我們直接跳過不看
if (connectable.getConnectableType() == ConnectableType.FUNNEL) {
// Handle Funnel as a special case because it will never be a 'source' component,
// and also its outgoing connections can not be terminated.
// Incoming FlowFiles from other components, and at least one outgoing connection are required.
return connectable.hasIncomingConnection()
&& hasNonLoopConnection
&& !connectable.getConnections().isEmpty()
&& Connectables.flowFilesQueued(connectable);
}
// 判斷當前的組件是否是Source,也就是我們前面說的源組件。以下任意情況都可以說明此組件是源組件
// 1 當組件被TriggerWhenEmpty註解標註的時候(這個註解你可以理解爲上游Connection沒有FLowFIle也執行調度)
// 2 當組件上游沒有Connection的時候(沒有輸入FlowFIle的Connection)
// 3 當組件上游的所有Connection都是來自於這個組件自己(自己連自己的一個循環),沒有來自其他組價的Connection
final boolean isSourceComponent = connectable.isTriggerWhenEmpty()
// No input connections
|| !connectable.hasIncomingConnection()
// Every incoming connection loops back to itself, no inputs from other components
|| !hasNonLoopConnection;
// 如果這個組件是源組件,那麼系統認爲它是有工作要做的,否則得看上游Connection是否有FlowFIle
return isSourceComponent || Connectables.flowFilesQueued(connectable);
}
ConnectableTask
的invoke()
方法經過一系列驗證如果判斷可以運行,那麼後面會有一行connectable.onTrigger(processContext, activeSessionFactory);
,就是調用Processor.onTrigger的入口。
總結一下,在Time drven
策略下(也包括CRON driven
),在觸發組件的onTrigger方法之前會先判斷這個這個組件是否有工作要做,判斷結果爲true纔會繼續運行下去,進而調用Processor的onTrigger方法。
疑問5
在理解maxThread設置一文中介紹了最大計時器線程計數(Max Timer Driven Thread Count)
,那麼問題是:在上面所述,我們檢測到組件沒有工作要做,在檢測的時候它佔用了線程池裏的線程嘛?
先說結論,檢測當前組件是否有工作可做的行爲是線程池分配一個線程來執行這次調度的一部分,是在調度最前面執行的邏輯,如果判斷沒有工作可做,那麼就不用繼續往下執行了,本次調度就結束了(本次調度不執行Processor的onTrigger方法),這個動作是很快很快很快的。也就是說檢測是否有工作可做的時候,是要佔用一個最大計時器線程計數(Max Timer Driven Thread Count)
裏的線程的。咋一聽,不太合理啊。我就配置了這麼多數量的線程池,你不幹活還得佔用我的一個線程。。。然而事實如此,下面我們分別從源碼和動手驗證來看一下:
首先NIFI繼承ScheduledThreadPoolExecutor
自定義了一個線程池FlowEngine(Timer Drriven和CRON Driven對應的是同一個線程池->FlowEngine實例,而Event Driven對應的是另一個線程池->FlowEngine實例),
public final class FlowEngine extends ScheduledThreadPoolExecutor
關於ScheduledThreadPoolExecutor這裏簡單說一下,ScheduledThreadPoolExecutor繼承了ThreadPoolExecutor,也就是說ScheduledThreadPoolExecutor擁有execute()和submit()提交異步任務的基礎功能,它的構造方法實際上是調用了ThreadPoolExecutor,ScheduledThreadPoolExecutor實現了ScheduledExecutorService接口,該接口定義了可延時執行異步任務和可週期執行異步任務的特有功能.而延時執行異步任務和週期執行異步任務分別就是Apache NIFI
CRON driven
和Timer driven
策略的實現核心所在。
//達到給定的延時時間後,執行任務。這裏傳入的是實現Runnable接口的任務,
//因此通過ScheduledFuture.get()獲取結果爲null
public ScheduledFuture<?> schedule(Runnable command,
long delay, TimeUnit unit);
//達到給定的延時時間後,執行任務。這裏傳入的是實現Callable接口的任務,
//因此,返回的是任務的最終計算結果
public <V> ScheduledFuture<V> schedule(Callable<V> callable,
long delay, TimeUnit unit);
//是以上一個任務開始的時間計時,period時間過去後,
//檢測上一個任務是否執行完畢,如果上一個任務執行完畢,
//則當前任務立即執行,如果上一個任務沒有執行完畢,則需要等上一個任務執行完畢後立即執行
public ScheduledFuture<?> scheduleAtFixedRate(Runnable command,
long initialDelay,
long period,
TimeUnit unit);
//當達到延時時間initialDelay後,任務開始執行。上一個任務執行結束後到下一次
//任務執行,中間延時時間間隔爲delay。以這種方式,週期性執行任務。
public ScheduledFuture<?> scheduleWithFixedDelay(Runnable command,
long initialDelay,
long delay,
TimeUnit unit);
以上是ScheduledThreadPoolExecutor執行延時和週期任務的方法,而在前面提到的TimerDrivenSchedulingAgent
就會調用flowEngine.scheduleWithFixedDelay
方法來週期的執行任務(對應疑問1每次任務執行完後再計時)。
TimerDrivenSchedulingAgent
類:
@Override
public void doSchedule(final Connectable connectable, final LifecycleState scheduleState) {
final List<ScheduledFuture<?>> futures = new ArrayList<>();
final ConnectableTask connectableTask = new ConnectableTask(this, connectable, flowController, contextFactory, scheduleState, encryptor);
for (int i = 0; i < connectable.getMaxConcurrentTasks(); i++) {
// 確定要運行的任務並創建它。
final AtomicReference<ScheduledFuture<?>> futureRef = new AtomicReference<>();
final Runnable trigger = createTrigger(connectableTask, scheduleState, futureRef);
// 週期性的去執行這個trigger
final ScheduledFuture<?> future = flowEngine.scheduleWithFixedDelay(trigger, 0L,
connectable.getSchedulingPeriod(TimeUnit.NANOSECONDS), TimeUnit.NANOSECONDS);
// now that we have the future, set the atomic reference so that if the component is yielded we
// are able to then cancel this future.
futureRef.set(future);
// Keep track of the futures so that we can update the ScheduleState.
futures.add(future);
}
scheduleState.setFutures(futures);
logger.info("Scheduled {} to run with {} threads", connectable, connectable.getMaxConcurrentTasks());
}
然後重點看上面代碼createTrigger(connectableTask, scheduleState, futureRef)
的實現:
private Runnable createTrigger(final ConnectableTask connectableTask, final LifecycleState scheduleState, final AtomicReference<ScheduledFuture<?>> futureRef) {
final Connectable connectable = connectableTask.getConnectable();
final Runnable yieldDetectionRunnable = new Runnable() {
@Override
public void run() {
// 調用上面說的connectableTask的invoke方法(上面提到過,這裏會判斷組件是否有工作要做等等邏輯)
final InvocationResult invocationResult = connectableTask.invoke();
if (invocationResult.isYield()) {
logger.debug("Yielding {} due to {}", connectable, invocationResult.getYieldExplanation());
}
// 如果組件yielded, 則取消任務重新提交,然後過了yield時間再運行。
//(這裏說的yield有兩種,一種是因爲組件異常,一種是因爲沒有工作可做,我們重點關注第二種)
final long newYieldExpiration = connectable.getYieldExpiration();
final long now = System.currentTimeMillis();
//組件處理數據發生異常(指沒有被捕獲已處理的那些)時,組件會被懲罰一段時間不再被調度。顯然沒有工作可做的組件不是這種情況
if (newYieldExpiration > now) {
...
} //noWorkYieldNanos就是上面疑問2說的10ms
else if (noWorkYieldNanos > 0L && invocationResult.isYield()) {
// Component itself didn't yield but there was no work to do, so the framework will choose
// to yield the component automatically for a short period of time.
final ScheduledFuture<?> scheduledFuture = futureRef.get();
if (scheduledFuture == null) {
return;
}
if (scheduledFuture.cancel(false)) {
synchronized (scheduleState) {
if (scheduleState.isScheduled()) {
//設置延遲10ms再調度一次
final ScheduledFuture<?> newFuture = flowEngine.scheduleWithFixedDelay(this, noWorkYieldNanos,
connectable.getSchedulingPeriod(TimeUnit.NANOSECONDS), TimeUnit.NANOSECONDS);
scheduleState.replaceFuture(scheduledFuture, newFuture);
futureRef.set(newFuture);
}
}
}
}
}
};
return yieldDetectionRunnable;
}
動手驗證的思路很簡單,我們在NIFI上有一些正在運行但沒有處理數據的Processor(沒有active task),然後修改一些代碼使得能夠我們查出來這個TimeDriven的FlowEngine線程池裏大概正在運行的線程數,如果爲0,那麼說明檢測
組件有沒有工作這個動作是不佔線程池資源,如果不爲0,那就是印證了上面我們分析源碼得到的結論,佔用線程池的資源。
在NIFI中我們設置有且只有4個正在運行的但不處理數據的Processor,如圖:
按照下面四個步驟修改一下代碼:
- 我們需要添加一個Rest接口幫助我們查詢線程池正在運行的大概的線程數,
ControllerResource
類增加
@GET
@Consumes(MediaType.WILDCARD)
@Produces(MediaType.APPLICATION_JSON)
@Path("activeThreads")
public int getActiveThreads() {
return flowController.timerDrivenEngineRef.get().getActiveCount();
}
- 將
ApplicationResource
中的flowController修改成public
public FlowController flowController;
- 將
FlowController
中的timerDrivenEngineRef修改成public
public final AtomicReference<FlowEngine> timerDrivenEngineRef;
- 防止線程運行太快,我們查不到正在運行的線程,我們在
TimerDrivenSchedulingAgent
類中處理沒有工作可做時的邏輯中添加Thread.sleep代碼
if (noWorkYieldNanos > 0L && invocationResult.isYield()) {
logger.info("當前組件沒有工作可以做,進入當前處理邏輯,防止處理太快我們查不到正在運行的線程,我們睡眠2秒");
try {
Thread.sleep(2000L);
} catch (InterruptedException e) {
e.printStackTrace();
}
// Component itself didn't yield but there was no work to do, so the framework will choose
// to yield the component automatically for a short period of time.
...
編譯後運行日誌截圖如下:
使用PostMan做Get請求結果如圖:
如果我們不按第四步修改代碼,日誌會很快很快的打印當前組件沒有工作可以做,進入當前處理邏輯,防止處理太快我們查不到正在運行的線程,我們睡眠2秒
,並且我們通過Rest接口查詢的結果通常就是0
源碼分析與動手驗證都證實了我們之前的結論,當這個組件啓動但是沒有處理數據,檢測這個組件有沒有工作可做也是佔用線程池的一部分資源的。
額外說一點,基於此疑問及得出的結論,我們應該知道,在NIFI中那些不再被使用到的流程和組件應該及時關閉或者清理掉。螞蟻再小也是肉,積累多了也是會佔用我們的資源的,也是基於這個原因,官方試圖通過新的Event Driven策略來彌補這些不足。
疑問6
如果組件週期運行不是0 sec
,我們假設是1000 sec
,那麼檢測
組件是否有工作的動作的頻率還是10ms嘛?
從上面提到的源碼分析,首先調度週期時間是由上一次任務運行完畢後再開始計時(調度週期偏向於FlowFIle之間流向組件的間隔),並且在檢測組件沒有工作要做後的再註冊的initialDelay還是10ms,那麼檢測
組件是否有工作的動作的頻率應該還是10ms。
動手驗證很簡單,基於疑問5搭建的驗證場景,我們將四個運行的組件變成一個,這個組件運行的週期設置爲1000 sec
。然後我們觀察日誌,如果日誌輸出的多條日誌之間是1000多秒,那麼證明我們上面說的結論是錯誤的,如果日誌之間是2秒多,那麼說明檢測
組件是否有工作的動作的頻率應該還是10ms。
總結一下:檢測組件是否有工作的動作的週期是與組件配置的週期時間無關的,默認的就是10ms
CRON driven
在技術實現上,它會先計算CRON表達式,得出現在的時間到定義的CRON下一次運行的時間點的時間差,然後調用的實質是JavaScheduledThreadPoolExecutor
本身提供的延時執行策略。
CRON driven和Timer driven使用的同一個FlowEngine線程池,同時受最大計時器線程計數(Max Timer Driven Thread Count)
控制。
CRON driven的調度配置是由六個必填字段和一個可選字段組成的字符串,每個字段之間用空格分隔。這些字段是:
Field | Valid values |
---|---|
Seconds | 0-59 |
Minutes | 0-59 |
Hours | 0-23 |
Day of Month | 1-31 |
Month | 1-12 or JAN-DEC |
Day of Week | 1-7 or SUN-SAT |
Year (optional) | empty, 1970-2099 |
通常,我們可以通過以下方式指定值:
-
Number: 指定一個或多個有效值。可以使用逗號分隔的列表輸入多個值。
-
Range: 使用
<number>-<number>
語法指定範圍。 -
Increment: 使用
<start value> /<increment>
語法指定一個增量。 比如, 配置Minutes, 0/15 表示 0, 15, 30, 45.
我們還應該注意幾個有效的特殊字符:
-
* — 指示所有值對該字段均有效。
-
? — 表示未指定特定值。該特殊字符在
Day of Month
和Day of Week
字段中有效。 -
L — 可以將L附加到
Day of Week
值後面,以指定該天是本月的最後一次出現。例如,1L表示該月的最後一個星期日。
比如:
-
0 0 13 * * ?
每天下午一點執行 -
0 20 14 ? * MON-FRI
週一到週五每天下午2:20執行 -
0 15 10 ? * 6L 2011-2017
2011年到2017年每月最後一個星期五上午10:15執行。
疑問1
基於上面的動手實驗,如果中間的一個UpdateAttribute
設置的是CRON driven,那麼上面所說的檢測
組件是否有工作要做的動作還是每10ms一次嘛?
這裏澄清一點,這個疑問本身就是錯的。本組件是CRON策略,時間到了某時某刻,啓動調度,如果有數據處理(或者是生成數據、拉取數據)就運行,如果沒有工作要做,那就結束,等待下一次調度(比如等到明天的某時某刻再執行)。所以CRON driven就沒有10ms週期檢測組件是否有工作要處理這個動作。
疑問2
使用CRON策略,比如我們設置的是0 0/10 * * * ?
每小時中0、10分鐘、20分鐘、30分鐘、40分鐘、50分鐘時間執行,然後如果到了10分鐘時去調度,但是上一次執行還沒有結束怎麼辦?
我們看一下QuartzSchedulingAgent
類中的doSchedule方法(start一個Processor就會調用這個方法)
@Override
public synchronized void doSchedule(final Connectable connectable, final LifecycleState scheduleState) {
final List<AtomicBoolean> existingTriggers = canceledTriggers.get(connectable);
if (existingTriggers != null) {
throw new IllegalStateException("Cannot schedule " + connectable + " because it is already scheduled to run");
}
//解析CRON表達式
final String cronSchedule = connectable.getSchedulingPeriod();
final CronExpression cronExpression;
try {
cronExpression = new CronExpression(cronSchedule);
} catch (final Exception pe) {
throw new IllegalStateException("Cannot schedule " + connectable + " to run because its scheduling period is not valid");
}
final List<AtomicBoolean> triggers = new ArrayList<>();
// 根據`Concurrent tasks`的配置,循環註冊任務
for (int i = 0; i < connectable.getMaxConcurrentTasks(); i++) {
final ConnectableTask continuallyRunTask = new ConnectableTask(this, connectable, flowController, contextFactory, scheduleState, encryptor);
final AtomicBoolean canceled = new AtomicBoolean(false);
//計算下一次CRON指定的時間點
final Date initialDate = cronExpression.getTimeAfter(new Date());
//計算現在的時間點到下一次CRON指定執行的時間點的時間差
final long initialDelay = initialDate.getTime() - System.currentTimeMillis();
final Runnable command = new Runnable() {
private Date nextSchedule = initialDate;
//達到這個時間點,虛擬機會運行run方法
@Override
public void run() {
if (canceled.get()) {
return;
}
try {
//這裏是上面提過的ConnectableTask的invoke方法,如果應該運行,就會調用Processor的onTrigger方法
continuallyRunTask.invoke();
} catch (final RuntimeException re) {
throw re;
} catch (final Exception e) {
throw new ProcessException(e);
}
if (canceled.get()) {
return;
}
//本次調度的主要工作做完了之後,再註冊下一次延遲執行
//計算下一次任務執行的延時時間
nextSchedule = getNextSchedule(nextSchedule, cronExpression);
final long delay = getDelay(nextSchedule);
logger.debug("Finished task for {}; next scheduled time is at {} after a delay of {} milliseconds", connectable, nextSchedule, delay);
//註冊下一次延時執行的任務
flowEngine.schedule(this, delay, TimeUnit.MILLISECONDS);
}
};
//註冊首次的延遲執行任務
flowEngine.schedule(command, initialDelay, TimeUnit.MILLISECONDS);
triggers.add(canceled);
}
canceledTriggers.put(connectable, triggers);
logger.info("Scheduled {} to run with {} threads on schedule {}", connectable, connectable.getMaxConcurrentTasks(), cronSchedule);
}
重要的是看一下getNextSchedule
這個方法,
private static Date getNextSchedule(final Date currentSchedule, final CronExpression cronExpression) {
// 由於時鐘的精度不是毫秒級,因此我們必須檢查是否安排了應該在該時間之後運行的下一個時間,否則我們可能最終會運行兩次相同的任務
final Date now = new Date();
return cronExpression.getTimeAfter(now.after(currentSchedule) ? now : currentSchedule);
}
由上面源碼我們來分析,比如我們設置的是0 0/10 * * * ?
每小時中0、10分鐘、20分鐘、30分鐘、40分鐘、50分鐘時間執行,然後第0分鐘運行了一個任務,它執行了15分鐘,在它做完所有工作後其實計算的是到第20分鐘這個時間點的延遲時間。
手動驗證一下,設置0/5 * * * * ?
,每分鐘每0秒、0+5…執行一次,複用上面的groovy代碼。
// 打印警告日誌
log.warn('我被調度了,本次調度發生時間爲'+String.valueOf(System.currentTimeMillis()))
//創建一個流文件
flowFile = session.create()
//添加一個屬性,在FlowFIle中記錄一個時間,姑且把這個時間當做本次調度開始時間
flowFile = session.putAttribute(flowFile, 'Time', String.valueOf(System.currentTimeMillis()))
//然後休眠10秒
try {
Thread.sleep(10000);
} catch (InterruptedException e) {
e.printStackTrace();
}
//將FlowFIle傳輸到success
session.transfer(flowFile, REL_SUCCESS)
我們得到4個流文件,一下依次是每個流文件的Time屬性值
1591608805094
+ 15秒
1591608820004
+ 15秒
1591608835002
+ 15秒
1591608850005
第0秒第一次調度,如果執行的很快,那麼第5秒應該執行第二次調度。但是第一次調度需要處理的數據很多,處理了10秒才完成任務,這個時候已經過了第5秒、第10秒這兩個時間點,所以計算延遲時間使用的是第15秒這個時間點。而最終我們看到的各個流文件Time這個值的間隔是15秒,而不是5秒。
Event driven
如果選擇此模式,則由event觸發處理器運行,並且該event在FlowFiles進入到該處理器上游Connection時發生。該模式當前被認爲是實驗性的,並非所有處理器都支持
(Processor類使用@EventDriven註解標註的Processor都支持Event driven)。選擇此模式後,Run schedule
選項不可配置,因爲不會觸發處理器定期
運行,而是由event觸發。此外,這是可以將Concurrent tasks
選項設置爲0的唯一模式。在這種情況下,線程數僅受Event-Driven Thread Pool
的大小限制。
疑問1
Concurrent tasks
配置還有效果嘛?
如果我們選擇了Event driven
,我們會看到Concurrent tasks
的默認值變成了0.這裏我們測試配置Concurrent tasks
爲2會不會生效。
我們看到Concurrent tasks
配置成2是有效的。
總結
Apache NIFI 內部對組件的調用提供了三種調度策略:Timer driven,CRON driven,Event driven。
其中我們日常使用的是Timer driven,CRON driven。他們共同使用一個線程池,共同受最大計時器線程計數(Max Timer Driven Thread Count)
限制。
Timer driven本質是使用ScheduledThreadPoolExecutor
的週期執行任務,CRON driven本質是使用ScheduledThreadPoolExecutor
延時執行任務。
Event driven是實驗性質的,目前也並不是所有的組件都支持,不建議使用,但我們應該知道,Event driven的想法是更好的,效率和資源使用是更優的,後期我們可以期待一下它的完成情況(當然也有可能完不成)
以上並沒有說明爲什麼Timer driven和CRON driven使用的是同一個線程池,這裏簡單說明一下,在FlowController
構造函數裏,quartzSchedulingAgent和timerDrivenAgent使用的是同一個FlowEngine實例
final QuartzSchedulingAgent quartzSchedulingAgent = new QuartzSchedulingAgent(this, timerDrivenEngineRef.get(), repositoryContextFactory, encryptor);
final TimerDrivenSchedulingAgent timerDrivenAgent = new TimerDrivenSchedulingAgent(this, timerDrivenEngineRef.get(), repositoryContextFactory, encryptor, this.nifiProperties);
公衆號
關注公衆號 得到第一手文章/文檔更新推送。