@InterfaceAudience.Public @InterfaceStability.Unstable
IOStatistics
API 的目的是提供個別 IO 類別(例如輸入和輸出串流)的統計資料,以一種標準的方式,讓應用程式可以查詢
許多與檔案系統相關的類別都已實作統計資料收集,並提供私人/不穩定的方式來查詢這些資料,但由於這些方式在不同的實作中並不相同,因此應用程式無法安全地參考這些值。範例:S3AInputStream
及其統計資料 API。這用於內部測試,但無法在應用程式(例如 Apache Hive 或 Apache HBase)的下游使用。
IOStatistics API 的目的是
hadoop-common
JAR 中具有基礎介面和類別。任何類別都可以實作 IOStatisticsSource
以提供統計資料。
包裝器 I/O 類別,例如 FSDataInputStream
和 FSDataOutputStream
應該實作介面,並將其轉發到包裝類別(如果它們也實作介面),否則傳回 null
。
IOStatisticsSource
實作的 getIOStatistics()
會傳回 IOStatistics
的執行個體,列舉該特定執行個體的統計資料。
IOStatistics
介面會匯出五種統計資料
類別 | 類型 | 說明 |
---|---|---|
計數器 |
長整數 |
一個可能會增加值的計數器;應大於或等於 0 |
量規 |
長整數 |
一個可以向上或向下的任意值;應大於或等於 0 |
最小值 |
長整數 |
一個最小值;可以為負值 |
最大值 |
長整數 |
一個最大值;可以為負值 |
平均統計資料 |
平均統計資料 |
一個算術平均值和樣本大小;平均值可以為負值 |
四個是簡單的 long
值,變異在於它們可能如何變更以及如何彙總。
對於不同的統計資料類別,aggregate(x, y)
的結果是
類別 | 彙總 |
---|---|
計數器 |
max(0, x) + max(0, y) |
量規 |
max(0, x) + max(0, y) |
最小值 |
min(x, y) |
最大值 |
max(x, y) |
平均統計資料 |
計算 x 和 y 的平均值 ) |
MeanStatistic
org.apache.hadoop.fs.statistics
此套件包含應用程式預計使用的公開統計資料 API。
MeanStatistic
是 (mean, samples)
的一個組,用於支援彙總。
樣本為 0
的 MeanStatistic
會被視為空的統計資料。
所有 sample = 0
的 MeanStatistic
執行個體都被視為相等,與 mean
值無關。
計算平均值的演算法
if x.samples = 0: y else if y.samples = 0 : x else: samples' = x.samples + y.samples mean' = (x.mean * x.samples) + (y.mean * y.samples) / samples' (samples', mean')
隱含地,這表示如果兩個樣本都為空,則彙總值也會為空。
public final class MeanStatistic implements Serializable, Cloneable { /** * Arithmetic mean. */ private double mean; /** * Number of samples used to calculate * the mean. */ private long samples; /** * Get the mean value. * @return the mean */ public double getMean() { return mean; } /** * Get the sample count. * @return the sample count; 0 means empty */ public long getSamples() { return samples; } /** * Is a statistic empty? * @return true if the sample count is 0 */ public boolean isEmpty() { return samples == 0; } /** * Add another mean statistic to create a new statistic. * When adding two statistics, if either is empty then * a copy of the non-empty statistic is returned. * If both are empty then a new empty statistic is returned. * * @param other other value * @return the aggregate mean */ public MeanStatistic add(final MeanStatistic other) { /* Implementation elided. */ } @Override public int hashCode() { return Objects.hash(mean, samples); } @Override public boolean equals(final Object o) { if (this == o) { return true; } if (o == null || getClass() != o.getClass()) { return false; } MeanStatistic that = (MeanStatistic) o; if (this.isEmpty()) { return that.isEmpty(); } return Double.compare(that.mean, mean) == 0 && samples == that.samples; } @Override public MeanStatistic clone() { return new MeanStatistic(this); } public MeanStatistic copy() { return new MeanStatistic(this); } }
org.apache.hadoop.fs.statistics.IOStatisticsSource
/** * A source of IO statistics. * These statistics MUST be instance specific, not thread local. */ @InterfaceStability.Unstable public interface IOStatisticsSource { /** * Return a statistics instance. * It is not a requirement that the same instance is returned every time. * {@link IOStatisticsSource}. * If the object implementing this is Closeable, this method * may return null if invoked on a closed object, even if * it returns a valid instance when called earlier. * @return an IOStatistics instance or null */ IOStatistics getIOStatistics(); }
這是物件執行個體如果為 IOStatistics 資訊來源,則必須實作的介面。
getIOStatistics()
的結果必須為下列之一
null
IOStatistics
。IOStatistics
的一個實例,其統計資料必須是實作 IOStatisticsSource
類別的該實例所獨有的。較不正式地說:如果傳回的統計資料對應不為空,則所有統計資料都必須從目前的實例收集,而不是從其他實例收集,就像收集某些 FileSystem
統計資料的方式。
getIOStatistics()
的結果(如果非空)在每次呼叫時都可能是不同的實例。
org.apache.hadoop.fs.statistics.IOStatistics
這些是實作 IOStatisticsSource
的物件提供的每個實例統計資料。
@InterfaceAudience.Public @InterfaceStability.Unstable public interface IOStatistics { /** * Map of counters. * @return the current map of counters. */ Map<String, Long> counters(); /** * Map of gauges. * @return the current map of gauges. */ Map<String, Long> gauges(); /** * Map of minumums. * @return the current map of minumums. */ Map<String, Long> minumums(); /** * Map of maximums. * @return the current map of maximums. */ Map<String, Long> maximums(); /** * Map of meanStatistics. * @return the current map of MeanStatistic statistics. */ Map<String, MeanStatistic> meanStatistics(); }
統計資料的命名原則設計為可讀、可共用,並理想上在所有 IOStatisticSource
實作中都一致。
金鑰名稱中的字元必須符合正規表示法 [a-z|0-9|_]
,但第一個字元除外,第一個字元必須在範圍 [a-z]
中。因此,有效統計資料名稱的完整正規表示法為
[a-z][a-z|0-9|_]+
在可能的情況下,統計資料的名稱應該使用定義為常見名稱的名稱。
org.apache.hadoop.fs.statistics.StreamStatisticNames org.apache.hadoop.fs.statistics.StoreStatisticNames
註 1:這些名稱正在演進中;為了讓客戶端安全地按名稱參照其統計資料,它們應該複製到應用程式中。(例如,對於編譯 hadoop 3.4.2 以連結到 hadoop 3.4.1 的應用程式,請複製字串)。
註 2:在這些類別中定義的金鑰不得從後續的 Hadoop 版本中移除。
常見的統計資料名稱不得用於報告任何其他統計資料,並且必須使用預先定義的測量單位。
其中一個對應中的統計資料名稱不應該在另一個對應中重複使用。這有助於診斷記錄的統計資料。
對於傳回的每個統計資料對應
不支援新增/移除條目的操作:傳回的對應可能由統計資料來源變更。
對應可能為空。
對應金鑰各代表一個已測量的統計資料。
對應中的金鑰組應該保持不變,並且不得移除金鑰。
統計資料應該動態:每次查詢條目都應該傳回最新值。
值可能在呼叫 Map.values()
和 Map.entries()
時變更
更新可能在傳回的迭代器的 iterable()
呼叫中,或可能在實際的 iterable.next()
操作中。也就是說:無法保證評估何時發生。
傳回的 Map.Entry
實例必須在重複的 getValue()
呼叫中傳回相同的值。(也就是說,一旦取得條目,它就是不可變的)。
統計資料查詢應該快速且非封鎖,如果在長時間操作期間呼叫,它們會優先傳回快速值,而不是最及時的值。
統計資料可能落後;特別是對於在個別操作中收集的統計資料(例如檔案系統實例提供的串流 IO 統計資料)。
代表時間的統計資料應該使用毫秒作為其單位。
代表時間且使用不同單位的統計資料必須記錄所使用的單位。
IOStatistics
的執行個體可以在執行緒間共用;
對提供的統計資料圖表進行讀取存取時,執行緒必須安全。
從圖表傳回的迭代器不能在執行緒間共用。
收集的統計資料必須包含所有執行緒在執行監控物件工作時執行的所有作業。
回報的統計資料不能僅限於主動執行緒。
這不同於 FileSystem.Statistics
行為,其中會收集和回報每個執行緒的統計資料。
該機制支援收集共用相同 FS 執行個體的不同工作執行緒的有限讀寫統計資料,但由於收集是執行緒本地的,因此它總是低報工作執行緒代表其他執行緒執行的 IO。
可以透過呼叫 IOStatisticsSupport.snapshotIOStatistics()
取得目前統計資料值的快照
public static <X extends IOStatistics & Serializable> X snapshotIOStatistics(IOStatistics statistics)
此快照可透過 Java 序列化和透過 Jackson 從/到 JSON 進行序列化。
org.apache.hadoop.fs.statistics.IOStatisticsSupport
此類別提供輔助方法來處理 IOStatistics 來源和執行個體。
請參閱其操作的 javadoc。
org.apache.hadoop.fs.statistics.IOStatisticsLogging
支援有效記錄 IOStatistics
/IOStatisticsSource
執行個體。
這些類別旨在協助記錄,包括僅在記錄層級需要時列舉 IOStatistics
執行個體的狀態。
LOG.info("IOStatistics after upload: {}", demandStringify(iostats)); // or even better, as it results in only a single object creations Object latest = demandStringify(iostats); LOG.info("IOStatistics : {}", latest); /* do some work. */ LOG.info("IOStatistics : {}", latest);
org.apache.hadoop.fs.statistics.impl
此套件包含實作類別,用於支援向應用程式提供統計資料。
應用程式不能使用這些類別。如果需要此套件中的功能,則可以透過 Hadoop 開發管道提出提供公開實作。
這些類別可以由 Hadoop FileSystem
、AbstractFileSystem
和相關類別的實作使用,這些類別不在 hadoop 原始程式碼樹中。實作人員必須注意,此程式碼的實作並不穩定,可能會在 Hadoop 的次要版本中變更。