微博數倉數據延時優化方案

前言

本文以離線數據倉庫爲背景,重點介紹因傳輸鏈路數據延時的不確定性,可能導致部分延遲文件無法參與正常的調度計算或同步,導致數據缺失的問題成因、業務影響及相應的解決方案。

關於這類問題的處理,有這麼一種論調:我們認爲正常情況下,缺失 數據的比例是很小的,可以大致認爲數據是可用的的;或者我們可以推後一下計算的時間,讓數據儘可能的傳輸完整;諸如此類…。

如果認可這種論調,可以直接忽略本文的內容。

我們是一個有 態度 的數據團隊,旨在精確評估用戶(整體/個體)的性能或行爲情況,以優質的數據驅動業務優化,數據必須做到客觀條件下最大限度地精準。

數倉架構

數據倉庫使用 Hive 構建,日誌或數據以文件形式(Text/ORCFile)存儲於HDFS。數倉整體劃分以下3層:

  • ODS(面向原始日誌的數據表)
  • DW(面向業務主題的數據表)
  • DM(面向業務應用的數據表)

日誌來源

日誌(原始日誌)來源可以是多樣的:

  • 實時數據流(Kafka/Flume/Scribe)
  • 離線數據推送(Rsync)
  • 日誌接口(Http/Wget)
  • 其它

無論使用哪一種方式,都會使用統一的目錄規範存儲於HDFS,如下:

${BASE_DIR}/業務日誌名稱/日期(yyyy_MM_dd)/小時(HH)/日誌文件名稱(帶有時間戳)

假設業務日誌名稱爲 www_spoollxrsaansnq8tjw0_aliyunXweibo,存儲目錄結構示例:

業務日誌目錄:
${BASE_DIR}/www_spoollxrsaansnq8tjw0_aliyunXweibo

日期目錄:
${BASE_DIR}/www_spoollxrsaansnq8tjw0_aliyunXweibo/2020_06_22
${BASE_DIR}/www_spoollxrsaansnq8tjw0_aliyunXweibo/2020_06_23
${BASE_DIR}/www_spoollxrsaansnq8tjw0_aliyunXweibo/2020_06_24

小時目錄:
${BASE_DIR}/www_spoollxrsaansnq8tjw0_aliyunXweibo/2020_06_24/09
${BASE_DIR}/www_spoollxrsaansnq8tjw0_aliyunXweibo/2020_06_24/10
${BASE_DIR}/www_spoollxrsaansnq8tjw0_aliyunXweibo/2020_06_24/11

日誌文件:
${BASE_DIR}/www_spoollxrsaansnq8tjw0_aliyunXweibo/2020_06_24/11/202006241100-node-aliyun-al01.xiaoka.tv-www_spoollxrsaansnq8tjw0_aliyunXweibo.log
${BASE_DIR}/www_spoollxrsaansnq8tjw0_aliyunXweibo/2020_06_24/11/202006241100-node-aliyun-static.yizhibo.com-www_spoollxrsaansnq8tjw0_aliyunXweibo.log

可見,目錄規範設計的存儲最小時間粒度爲 小時。實時數據流使用定時(如:5分鐘)迭代新文件的方式存儲數據,存儲的具體日期/小時目錄由創建新文件時的 當前時間 決定;離線數據推送或日誌接口會從日誌文件名稱中提取時間戳,存儲到該時間戳對應的日期/小時目錄。

:需要留心不同來源的日誌存儲於HDFS的方式,後續會再次涉及到這部分內容。

數據表創建及分區動態掛載

爲業務日誌 www_spoollxrsaansnq8tjw0_aliyunXweibo 創建相應的原始日誌數據表(ODS),假設數據表名稱 aliyunXweibo :

CREATE EXTERNAL TABLE IF NOT EXISTS aliyunXweibo (
    col1 STRING,
    col2 STRING,
    col3 STRING
    ...
)
PARTITIONED BY (dt STRING)
ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t'
;

數據表 aliyunXweibo 類型爲外表(EXTERNAL),使用 dt 爲分區(Partition)字段,格式:yyyyMMddHH,與HDFS存儲目錄中的日期/小時相互對應,如:

dt = '2020062400'

對應着

${BASE_DIR}/www_spoollxrsaansnq8tjw0_aliyunXweibo/2020_06_24/00

分區動態掛載就是建立數據表分區與HDFS日期/小時目錄的 關係。我們部署有一套專用的服務,定時執行以下2個操作:

  1. 如果某個業務日誌的日期/小時目錄存在,但業務數據表中相應的分區不存在,則執行 掛載

    ALTER TABLE aliyunXweibo ADD PARTITION(dt = '2020062400') LOCATION '${BASE_DIR}/www_spoollxrsaansnq8tjw0_aliyunXweibo/2020_06_24/00';
    
  2. 如果某個業務數據表中的分區存在,但業務日誌中相應的日期/小時目錄不存在(過期被刪除),則執行 移除掛載

    ALTER TABLE aliyunXweibo DROP IF EXISTS PARTITION(dt = '2020062400');
    

數據計算

原始日誌數據表創建及分區掛載完成之後,便可以根據業務分析需求,開發部署相應的業務計算應用(Hive SQL/Spark SQL/Spark Application,DW/DM),計算結果以數據表(DW/DM)的形式存儲於數據倉庫中,這些數據表使用 dt 爲分區字段,時間粒度爲 小時

數據表分區掛載、DW數據計算應用及DM數據計算應用可以簡單理解爲使用 小時 爲時間週期定時(如:每小時第5分鐘)運行,調度及相互之間的依賴關係均由調度系統負責。

假設調度時間爲 2020-06-24 01:05:00,調度系統中的應用會掛載和計算 上一小時 的數據,數據表分區掛載及DW/DM數據計算應用會運行完成之後,各個數據表新生成的分區如下:

aliyunXweibo(ODS)
dt = '2020062400'

aliyunXweibo_DW(DW)
dt = '2020062400'

aliyunXweibo_DM(DM)
dt = '2020062400'

:實際場景中,這一步會比較複雜(也請留心),這裏僅用於理解流程,詳情見後文。

 
綜上所述,數據倉庫經過日誌存儲、數據表創建及分區掛載、數據計算等一系列相關的流程之後,最後會形成典型的 ODS/DW/DM 3層業務數據表,用以支撐業務不同場景下的應用。

數據同步

業務場景中,有一個很重要的應用部分就是數據可視化或數據接口,其顯著特點是要求 查詢響應耗時低 (如:1-3 s),數據來源通常是數據倉庫中的DW/DM數據表。如前文所述,數據倉庫使用 Hive 構建,鑑於 Hive 自身的設計及特點,無法滿足 查詢響應耗時低 這個要求。通常的做法是將需要查詢的數據表數據 同步 到合適的查詢引擎(如:ElasticSearch、ClickHouse、MySQL等),然後使用這些查詢引擎服務於數據可視化或數據接口。

數據同步 就是指將數據倉庫中數據表(DW/DM)的數據同步至適用於 **即席查詢(OLAP)**場景的查詢引擎中的過程;每次追加同步一個分區的數據。

也就是說,DW/DM數據計算應用運行完成之後,還有會相應的數據同步應用,負責同步DW/DM數據表中相應時間區分中數據至外部的查詢引擎。

繼續使用 數據計算 中的示例,假設需要同步的DW/DM數據表分區:

aliyunXweibo_DW(DW)
dt = '2020062400'

aliyunXweibo_DM(DM)
dt = '2020062400'

實際同步時會使用這樣的SQL:

SELECT
    *
FROM
    aliyunXweibo_DW
WHERE
    dt = '2020062400'

數據延時

前文中描述的數據倉庫/數據計算/數據同步,應該是目前業界普通採用的一種數據方案,實際工作場景中以此爲指導也可以運行的不錯。那麼,問題在哪裏?會影響什麼?

細節,細節,還是細節 !!!

以微博客戶端爲例,用戶使用過程中會產生大量的性能或行爲日誌,這些日誌因爲業務場景的差異,日誌上報的時機也是不同的。考慮到用戶體驗的問題,有一種常用的策略是 客戶端切換至後臺,且網絡連接Wifi,這時會將客戶端中的日誌上傳至服務端。服務端接收到日誌之後會中轉至 Kafka,供有需要的業務方消費使用。

:日誌中通常會包含時間戳,表示日誌記錄時間或用戶行爲時間。後文統一稱之爲 日誌時間

簡單思考就可以看出,客戶端切換後臺網絡連接Wifi 都是很不可控的因素,受用戶使用行爲或使用場景影響較大。比如:用戶於Wifi環境下連續刷微博1小時,我們可以認爲這1小時之內是沒有任何日誌上傳,只有等用戶使用完微博,將客戶端切換至後臺且連接Wifi的條件下日誌纔會上傳。也說是說,用戶這部分日誌被服務端接收到至少會延遲1小時。

客戶端記錄的 日誌時間 與服務端接收到日誌的 當前時間 之間的 時間差,就是 數據延時

數據延時 是絕對存在且不可避免的。這裏的 延時 特指超出正常傳輸本身所需時間之外的部分。

對於存儲的影響?

對於實時數據流,即使我們可以實時消費 (Kafka)中的數據,但數據本身可能是延時的,即日誌時間與消費的當前時間相比滯後。消費 數據存儲至HDFS時,可以有2種時間策略:

  1. 當前時間

    使用 當前時間 創建相應日期/小時目錄的文件,將之後 5分鐘 (假設文件迭代週期爲5分鐘)之內消費的數據寫入到該文件中;下一個 5分鐘 到來時,繼續使用 當前時間 創建新文件及寫入數據。

    影響:可能會導致當前的日期/小時目錄的文件被寫入 日誌。這裏的 ,是說從這些日誌的 日誌時間 來看,並不屬於當前的日期/小時目錄,可能屬於過去某個日期/小時目錄(數據延時),也可能屬於下一個日期/小時目錄(文件迭代不及時)。

  2. 日誌時間

    使用 日誌時間 創建相應日期/小時目錄的文件,將消費的數據按 日誌時間 寫入到特定文件中;已創建的文件如果 5分鐘 (假設文件迭代週期爲5分鐘)之內沒有新的數據寫入則關閉。

    影響:可能會導致過去的某個日期/小時目錄的文件被不定時的寫入日誌;也就是說,使用某個日期/小時目錄中的文件時,可能還有部分數據沒有完全傳輸完成。

 
對於離線數據推送,因爲使用日誌文件名稱中的時間戳決定寫入哪個日期/小時目錄,僅僅可能會導致類似於實時數據流使用 日誌時間 的影響,後續不再贅述。
 
存儲層的影響會 擴散 到計算層。

對於計算或同步的影響?

當我們想計算某個數據表 前1小時 的數據時,因爲數據時延及存儲層選用時間策略的關係,可能會遇到以下問題:

  1. 前1小時目錄中,可能包含不確定比例的 數據(當前時間);

  2. 前1小時目錄中,可能缺失不確定比例的數據(日誌時間);

  3. 存儲層使用 當前時間;

    計算ODS數據表 前1小時 分區數據時,因爲這個小時分區目錄中可能包含不屬於該小時的數據,保存(Insert)計算結果到DW數據表時實際會寫出多個小時分區,如:

    aliyunXweibo(ODS)
    dt = '2020062400'
    
    aliyunXweibo_DW(DW)
    dt = '2020062322'
    dt = '2020062323'
    dt = '2020062400'
    dt = '2020062401'
    

    其中,aliyunXweibo_DW(2020062322) 與 aliyunXweibo_DW(2020062323) 是因爲 aliyunXweibo(2020062400) 中包含有因數據延時導致的 日誌導致的;aliyunXweibo_DW(2020062401) 是因爲 aliyunXweibo(2020062400) 中包含有因文件迭代不及時的 超前 日誌導致的。注意,aliyunXweibo_DW(2020062322) 與 aliyunXweibo_DW(2020062323) 這2個分區在前2個調度的週期中已計算且同步完成,這次相當於是在已有分區中 追加 (Append) 部分數據。

    如前所述,這次同步 aliyunXweibo_DW(2020062400) 會使用SQL:

    SELECT
        *
    FROM
        aliyunXweibo_DW
    WHERE
        dt = '2020062400'
    

    很明顯,aliyunXweibo_DW(2020062311) 與 aliyunXweibo_DW(2020062312) 中新 追加 的數據是無法參與同步的,對於查詢引擎而言,這部分 追加 的數據是永遠不可見的,相當於 丟失。aliyunXweibo_DW(2020062401) 中的數據會被即將發生的 下一次 調度所同步。

  4. 存儲層使用 日誌時間

    計算ODS數據表 前1小時 分區數據時,保存(Insert)計算結果到DW數據表時僅會寫出1個小時分區,如:

    aliyunXweibo(ODS)
    dt = '2020062400'
    
    aliyunXweibo_DW(DW)
    dt = '2020062400'
    

    同步 aliyunXweibo_DW(2020062400) 會使用SQL:

    SELECT
        *
    FROM
        aliyunXweibo_DW
    WHERE
        dt = '2020062400'
    

    因爲計算時屬於這個小時分區目錄中的文件可能還沒有被全部傳輸完成,這部分文件永遠也不會參與後續的調度計算或同步,相當於 丟失

解決方案

最簡單的解決方案就是定時重新計算且更新(覆蓋)一定時間範圍內的歷史數據,考數據量級、資源成本及數據服務影響情況,相信大多數公司不會選擇這種方案。

思路

我們解決方案的核心是 增量增量計算增量同步

  1. 增量計算

    每一次調度計算 上一小時 的數據時,不是計算 上一小時分區目錄的文件,而是以增量的方式計算 上一小時新寫入的文件

  2. 增量同步

    每一次調度同步 上一小時 的數據時,不是同步 上一小時分區目錄的文件,而是以增量的方式同步 上一小時新寫入的文件

Hive 原生支持以全表或分區的方式讀取數據,我們需要進行相應地擴展使其支持:

讀取數據表指定時間圍內寫入的文件數據

:文件寫入時間使用文件最後修改時間(mtime)表示,可通過Hadoop FileSystem API獲取。

Hive SQL 不支持表述這樣的邏輯,假設 存在如下屬性:

  • dip.mapreduce.input.fileinputformat.sync:用於表示開啓自定義特性,默認不開啓。
  • dip.mapreduce.input.fileinputformat.starttime:用於表示文件最後修改時間的起始時間點,毫秒,閉區間。
  • dip.mapreduce.input.fileinputformat.endtime:用於表示文件最後修改時間的截止時間點,毫秒,閉區間。

如果需要讀取數據表 aliyunXweibo 於 [2020-06-24 00:00:00, 2020-06-24 00:59:59] 內寫入的文件數據,SQL如下:

set dip.mapreduce.input.fileinputformat.sync=true;

set dip.mapreduce.input.fileinputformat.starttime=1592928000000;
set dip.mapreduce.input.fileinputformat.endtime=1592931599000;

SELECT
    *
FROM
    aliyunXweibo

假設 成立的場景下,Hive 執行邏輯:

  1. 加載數據表 aliyunXweibo 目錄下的所有文件列表;
  2. 遍歷文件列表中的每一個文件,根據文件最後修改時間(mtime)過濾掉不在指定時間範圍內([1592928000000, 1592931599000])的文件;
  3. 使用過濾之後的文件列表,完成後續的流程;

上述方式會掃描全表的文件,如果考慮文件數目較多,可以結合業務情況約束時間分區範圍。

如果需要讀取數據表 aliyunXweibo,時間分區 [2020061701, 2020062401],時間範圍 [2020-06-24 00:00:00, 2020-06-24 00:59:59]內寫入的文件數據:

set dip.mapreduce.input.fileinputformat.sync=true;

set dip.mapreduce.input.fileinputformat.starttime=1592928000000;
set dip.mapreduce.input.fileinputformat.endtime=1592931599000;

SELECT
    *
FROM
    aliyunXweibo
WHERE
    dt >= '2020061701'
    AND dt <= '2020062401'

此時,Hive 執行邏輯:

  1. 加載數據表 aliyunXweibo 指定分區目錄下的文件列表;
  2. 遍歷文件列表中的每一個文件,根據文件最後修改時間(mtime)過濾掉不在指定時間範圍([1592928000000, 1592931599000])內的文件;
  3. 使用過濾之後的文件列表,完成後續的流程;

 
調度計算或同步需要讀取 上一小時 的數據時,只需要通過調度時間轉換爲 [starttime, endtime] 即可。

 
如上所述,我們是通過自定義Hive配置屬性的方式,使其支持按指定時間範圍以增量的方式讀取文件的。那麼,具體如何實現呢

Hive讀取文件是通過 Hadoop MapReduce InputFormat 實現的,以 TEXTFILE (默認) 爲例,實際使用的是 TextInputFormat(org.apache.hadoop.mapred.TextInputFormat),繼承自 FileInputFormat (org.apache.hadoop.mapred.FileInputFormat),重點介紹以下 3 個方法:

  • getInputPaths

    通過屬性 mapreduce.input.fileinputformat.inputdir 獲取輸入目錄,誰來設置這個屬性的值呢?
    Hive會解析SQL語句,從中提取出數據表名稱及分區過濾條件,動態計算需要讀取的HDFS目錄,然後將目錄值(多個)設置到屬性 mapreduce.input.fileinputformat.inputdir 中。
     

  • listStatus

    通過 getInputPaths() 獲取輸入目錄,獲取這些目錄中所有文件對應的文件狀態信息(FileStatus)。其中,文件狀態信息中包括文件最後修改時間(modification_time)。

    listStatus() 也支持自定義過濾器(PathFilter),用於根據文件路徑(Path)過濾文件,可以通過屬性 mapreduce.input.pathFilter.class 指定。
     

  • getSplits

    通過 listStatus() 獲取輸入文件,形成切片(Split)用於後續處理邏輯使用。

 
受 listStatus() 啓發,既然可以通過自定義過濾器過濾文件,我們也可以擴展代碼,獲取自定義屬性,然後根據文件最後修改時間過濾文件。

源碼擴展

以 Hadoop 2.8.2 爲例,在源文件 hadoop-2.8.2-src/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileInputFormat.java listStatus() 方法末尾處加入代碼(260 ~ 279):

254     sw.stop();
255     if (LOG.isDebugEnabled()) {
256       LOG.debug("Time taken to get FileStatuses: "
257           + sw.now(TimeUnit.MILLISECONDS));
258     }
259
260     boolean sync = job.getBoolean("dip.mapreduce.input.fileinputformat.sync", false);
261     if (sync && ArrayUtils.isNotEmpty(result)) {
262       long startTime = job.getLong("dip.mapreduce.input.fileinputformat.starttime", Long.MIN_VALUE);
263       long endTime = job.getLong("dip.mapreduce.input.fileinputformat.endtime", Long.MAX_VALUE);
264
265       result =
266           Arrays.stream(result)
267               .filter(
268                   file -> {
269                     boolean meet =
270                         startTime <= file.getModificationTime()
271                             && file.getModificationTime() <= endTime;
272                     if (meet) {
273                       LOG.info("Input meet path: " + file.getPath().toString());
274                     }
275
276                     return meet;
277                   })
278               .toArray(FileStatus[]::new);
279     }
280
281     LOG.info("Total input files to process : " + result.length);
282     return result;
283   }

代碼說明

  1. 獲取是否開啓自定義特性;

    boolean sync = job.getBoolean("dip.mapreduce.input.fileinputformat.sync", false);
    
  2. 如果開啓自定義特性(sync 爲 true)且輸入文件不爲空,則獲取文件寫入時間範圍,並執行過濾;

    獲取文件寫入起始時間(startTime)、截止時間(endTime);

    long startTime = job.getLong("dip.mapreduce.input.fileinputformat.starttime", Long.MIN_VALUE);
    long endTime = job.getLong("dip.mapreduce.input.fileinputformat.endtime", Long.MAX_VALUE);
    

    遍歷文件列表(result)中的每一個文件(file),判斷文件最後修改時間(file.getModificationTime())是否位於指定的文件寫入時間範圍([startTime, endTime]);

    startTime <= file.getModificationTime()
        && file.getModificationTime() <= endTime;
    

編譯部署

編譯

mvn clean package -Dmaven.test.skip=true -Djavac.version=1.8

使用編譯生成的Jar文件

hadoop-2.8.2-src/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/target/hadoop-mapreduce-client-core-2.8.2.jar

替換Hadoop部署目錄的Jar文件

hadoop-2.8.2/share/hadoop/mapreduce/hadoop-mapreduce-client-core-2.8.2.jar

:如果使用的是HiveServer2,需要重啓實例。

使用示例

假設我們需要統計數據表 mytable 指定文件寫入時間範圍 [1592915400000, 1592922600000] 內的行數。

Hive

set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;

set dip.mapreduce.input.fileinputformat.sync=true;

set dip.mapreduce.input.fileinputformat.starttime=1592915400000;
set dip.mapreduce.input.fileinputformat.endtime=1592922600000;

SELECT
    COUNT(1)
FROM
    mytable

:Hive 對於 InputFormat 有一些自定義的封裝,默認使用org.apache.hadoop.hive.ql.io.CombineHiveInputFormat,詳情參考相關文檔。

Spark

from pyspark.sql import SparkSession

if __name__ == '__main__':
    sql = '''
    SELECT
        COUNT(1)
    FROM
        mytable
    '''
    sql = sql.strip()

    session = SparkSession.builder \
        .enableHiveSupport() \
        .getOrCreate()

    session.sql('set dip.mapreduce.input.fileinputformat.sync=true')
    
    session.sql('set dip.mapreduce.input.fileinputformat.starttime=1592917200000')
    session.sql('set dip.mapreduce.input.fileinputformat.endtime=1592928000000')

    session.sql(sql).show()

    session.stop()

:測試時可能在數據表 mytable 位於HDFS的存儲目錄中放入不同時間(文件最後修改時間)的文件進行模擬。

特別說明

Hive 使用的 InputFormat 實現類有很多,且不是所有的實現類都繼承自 FileInputFormat,本文提供的只是一種通用的解決思案,實際使用時需要根據具體使用的 InputFormat 進行擴展。

結語

本文中討論的 數據缺失 問題,之前與同事的討論中曾一度被稱之爲 業界難題 進而被一度擱置。同時也可以看出,解決方案的實現過程雖然會涉及少量Hadoop源代碼的擴展,但本質是十分簡單的。問題的解決還是取決於工程師對技術及業務的掌控程度,僅供大家參考。

 
 
 
avatar

發表評論
所有評論
還沒有人評論,想成為第一個評論的人麼? 請在上方評論欄輸入並且點擊發布.
相關文章