Interface SupportsWritingMetadata


  • @PublicEvolving
    public interface SupportsWritingMetadata
    Interface for DynamicTableSinks that support writing metadata columns.

    Metadata columns add additional columns to the table's schema. A table sink is responsible for accepting requested metadata columns at the end of consumed rows and persist them. This includes potentially forwarding metadata columns to contained formats.

    Examples in SQL look like:

    
     // writes data to the corresponding metadata key `timestamp`
     CREATE TABLE t1 (i INT, s STRING, timestamp TIMESTAMP(3) WITH LOCAL TIME ZONE METADATA, d DOUBLE)
    
     // casts data from INT and writes to metadata key `timestamp`
     CREATE TABLE t2 (i INT, s STRING, myTimestamp INT METADATA FROM 'timestamp', d DOUBLE)
    
     // metadata is not persisted because metadata column is virtual
     CREATE TABLE t3 (i INT, s STRING, timestamp TIMESTAMP(3) WITH LOCAL TIME ZONE METADATA VIRTUAL, d DOUBLE)
     

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

    If this interface is implemented, listWritableMetadata() lists all metadata keys and their corresponding data types that the sink 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 and will call applyWritableMetadata(List, DataType) with a list of metadata keys. An implementation must ensure that metadata columns are accepted at the end of the physical row in the order of the provided list after the apply method has been called.

    The metadata column's data type must match with listWritableMetadata(). For the examples above, this means that a table sink for `t2` accepts a TIMESTAMP and not INT. The casting from INT will be performed by the planner in a preceding operation:

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

      • listWritableMetadata

        Map<String,​DataType> listWritableMetadata()
        Returns the map of metadata keys and their corresponding data types that can be consumed by this table sink for writing.

        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 applyWritableMetadata(List, DataType). Therefore, it might be beneficial to return a LinkedHashMap if a strict metadata column order is required.

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

        
         KEY FORMAT METADATA COLUMNS + VALUE FORMAT METADATA COLUMNS + SINK METADATA COLUMNS
         

        Metadata key names follow the same pattern as mentioned in Factory. In case of duplicate names in format and sink 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:
        EncodingFormat.listWritableMetadata()
      • applyWritableMetadata

        void applyWritableMetadata​(List<String> metadataKeys,
                                   DataType consumedDataType)
        Provides a list of metadata keys that the consumed RowData will contain as appended metadata columns which must be persisted.
        Parameters:
        metadataKeys - a subset of the keys returned by listWritableMetadata(), ordered by the iteration order of returned map
        consumedDataType - the final input type of the sink, it is intended to be only forwarded and the planner will decide on the field names to avoid collisions
        See Also:
        EncodingFormat.applyWritableMetadata(List)