使用 IOStatistics API 收集统计信息

@InterfaceAudience.Public
@InterfaceStability.Unstable

IOStatistics API 旨在以标准方式提供有关各个 IO 类的统计信息(例如输入和输出流),应用程序可以查询该统计信息

许多与文件系统相关的类都已实现统计信息收集,并提供了查询此信息的方式,但由于这些方式在各个实现中并不通用,因此应用程序引用这些值并不安全。示例:S3AInputStream 及其统计信息 API。这在内部测试中使用,但无法在 Apache Hive 或 Apache HBase 等应用程序的下游使用。

IOStatistics API 旨在

  1. 特定于实例,而不是在类的多个实例或线程本地之间共享。
  2. 公开且足够稳定,可供应用程序使用。
  3. 易于在用 Java、Scala 以及通过 libhdfs 用 C/C++ 编写的应用程序中使用
  4. hadoop-common JAR 中具有基础接口和类。

核心模型

任何类都可以实现 IOStatisticsSource 以提供统计信息。

诸如 FSDataInputStreamFSDataOutputStream 的包装器 I/O 类实现该接口并将其转发到包装类(如果它们也实现了该接口),否则返回 null

IOStatisticsSource 实现 getIOStatistics() 返回 IOStatistics 实例,枚举该特定实例的统计信息。

IOStatistics 接口导出五种统计信息

类别 类型 描述
计数器 长整型 值可能增加的计数器;应大于或等于 0
刻度 长整型 一个可以上下波动的任意值;应大于或等于 0
最小值 长整型 最小值;可以为负值
最大值 长整型 最大值;可以为负值
平均值统计信息 MeanStatistic 算术平均值和样本量;平均值可以为负值

有四个是简单的 long 值,变化方式和聚合方式可能有所不同。

统计值聚合

对于不同的统计类别,aggregate(x, y) 的结果是

类别 聚合
计数器 max(0, x) + max(0, y)
刻度 max(0, x) + max(0, y)
最小值 min(x, y)
最大值 max(x, y)
平均值统计信息 计算 xy 的平均值)

MeanStatistic

org.apache.hadoop.fs.statistics

此包包含供应用程序使用的公共统计信息 API。

MeanStatistic(mean, samples) 的元组,用于支持聚合。

样本为 0MeanStatistic 被视为一个空统计信息。

所有 sample = 0MeanStatistic 实例都被视为相等,与 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() 的结果非空,则在每次调用时都 MAY 为不同的实例。

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|_]+
    
  • 在可能的情况下,统计信息名称 SHOULD 为使用通用名称定义的名称。

    org.apache.hadoop.fs.statistics.StreamStatisticNames
    org.apache.hadoop.fs.statistics.StoreStatisticNames
    

注意 1:这些名称正在不断发展;为了让客户端安全地通过名称引用其统计信息,SHOULD 将其复制到应用程序中。(例如,对于编译 Hadoop 3.4.2 以链接到 Hadoop 3.4.1 的应用程序,复制这些字符串)。

注意 2:在这些类中定义的键 SHALL NOT 从后续 Hadoop 版本中删除。

  • MUST NOT 使用通用统计信息名称来报告任何其他统计信息,并且 MUST 使用预定义的测量单位。

  • SHOULD NOT 在一个映射中的统计信息名称在另一个映射中重复使用。这有助于诊断记录的统计信息。

统计信息映射

对于返回的每个统计信息映射

  • 不支持添加/删除条目的操作:返回的映射 MAY 可以通过统计信息源进行修改。

  • 映射 MAY 为空。

  • 映射键各自表示一个已测量的统计信息。

  • 映射中的键集 SHOULD 保持不变,并且 MUST NOT 删除键。

  • 统计信息 SHOULD 为动态的:每次查找条目 SHOULD 返回最新值。

  • 值 MAY 在 Map.values()Map.entries() 的调用中发生变化

  • 更新 MAY 在返回的迭代器的 iterable() 调用中,或 MAY 在实际的 iterable.next() 操作中。也就是说:无法保证评估何时发生。

  • 返回的 Map.Entry 实例 MUST 在重复的 getValue() 调用中返回相同的值。(即一旦获得条目,它就是不可变的)。

  • 统计信息查询 SHOULD 尽可能快速且非阻塞,如果在长时间操作期间调用,它们会优先快速返回而不是最及时返回。

  • 统计信息 MAY 滞后;特别是对于在单独操作中收集的统计信息(例如文件系统实例提供的流 IO 统计信息)。

  • 表示时间的统计信息 SHOULD 使用毫秒作为其单位。

  • 表示时间且使用不同单位的统计信息 MUST 记录所使用的单位。

线程模型

  1. IOStatistics 的实例可以在线程之间共享;

  2. 对提供的统计信息映射的读取访问 MUST 是线程安全的。

  3. 从映射中返回的迭代器 MUST NOT 在线程之间共享。

  4. 收集的统计信息 MUST 包括为受监视对象执行工作的各个线程执行的所有操作。

  5. 报告的统计数据不得局限于活动线程。

这与 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 FileSystemAbstractFileSystem 和相关类(这些类不在 hadoop 源代码树中)的实现。实现者必须意识到此代码的实现不稳定,并且可能会在 Hadoop 的次要版本中发生更改。