Interface SupportsReadingMetadata

  • All Known Implementing Classes:
    FileSystemTableSource

    @PublicEvolving
    public interface SupportsReadingMetadata
    Interface for ScanTableSources that support reading metadata columns.

    Metadata columns add additional columns to the table's schema. A table source is responsible for adding requested metadata columns at the end of produced rows. This includes potentially forwarding metadata columns from contained formats.

    Examples in SQL look like:

    
     // reads the column from corresponding metadata key `timestamp`
     CREATE TABLE t1 (i INT, s STRING, timestamp TIMESTAMP(3) WITH LOCAL TIME ZONE METADATA, d DOUBLE)
    
     // reads the column from metadata key `timestamp` and casts to INT
     CREATE TABLE t2 (i INT, s STRING, myTimestamp INT METADATA FROM 'timestamp', d DOUBLE)
     

    By default, if this interface is not implemented, the statements above would fail because the table source does not provide a metadata key called `timestamp`.

    If this interface is implemented, listReadableMetadata() lists all metadata keys and their corresponding data types that the source exposes to the planner. The planner will use this information for validation and insertion of explicit casts if necessary.

    The planner will select required metadata columns (i.e. perform projection push down) and will call applyReadableMetadata(List, DataType) with a list of metadata keys. An implementation must ensure that metadata columns are appended at the end of the physical row in the order of the provided list after the apply method has been called, e.g. using JoinedRowData.

    Note: The final output data type emitted by a source changes from the physically produced data type to a data type with metadata columns. applyReadableMetadata(List, DataType) will pass the updated data type for convenience. If a source implements SupportsProjectionPushDown, the projection must be applied to the physical data in the first step. The passed updated data type will have considered information from SupportsProjectionPushDown already.

    The metadata column's data type must match with listReadableMetadata(). For the examples above, this means that a table source for `t2` returns a TIMESTAMP and not INT. The casting to INT will be performed by the planner in a subsequent operation:

    
     // physical output
     ROW < i INT, s STRING, d DOUBLE >
    
     // final output (i.e. produced type) for t1
     ROW < i INT, s STRING, d DOUBLE, timestamp TIMESTAMP(3) WITH LOCAL TIME ZONE >
    
     // final output (i.e. produced type) for t2
     ROW < i INT, s STRING, d DOUBLE, myTimestamp TIMESTAMP(3) WITH LOCAL TIME ZONE >
     
    • Method Detail

      • listReadableMetadata

        Map<String,​DataType> listReadableMetadata()
        Returns the map of metadata keys and their corresponding data types that can be produced by this table source for reading.

        The returned map will be used by the planner for validation and insertion of explicit casts (see LogicalTypeCasts.supportsExplicitCast(LogicalType, LogicalType)) if necessary.

        The iteration order of the returned map determines the order of metadata keys in the list passed in applyReadableMetadata(List, DataType). Therefore, it might be beneficial to return a LinkedHashMap if a strict metadata column order is required.

        If a source forwards metadata from one or more formats, we recommend the following column order for consistency:

        
         KEY FORMAT METADATA COLUMNS + VALUE FORMAT METADATA COLUMNS + SOURCE METADATA COLUMNS
         

        Metadata key names follow the same pattern as mentioned in Factory. In case of duplicate names in format and source keys, format keys shall have higher precedence.

        Regardless of the returned DataTypes, a metadata column is always represented using internal data structures (see RowData).

        See Also:
        DecodingFormat.listReadableMetadata()
      • supportsMetadataProjection

        default boolean supportsMetadataProjection()
        Defines whether projections can be applied to metadata columns.

        This method is only called if the source does not implement SupportsProjectionPushDown. By default, the planner will only apply metadata columns which have actually been selected in the query regardless. By returning false instead the source can inform the planner to apply all metadata columns defined in the table's schema.

        If the source implements SupportsProjectionPushDown, projections of metadata columns are always considered before calling applyReadableMetadata(List, DataType).