You are viewing an old version of this page. View the current version.

Compare with Current View Page History

« Previous Version 5 Next »

Status

Current state"Under Discussion"

Discussion thread: https://lists.apache.org/thread/zjc4p47k4sxjcbnntt8od2grnmx51xg0

JIRAhere (<- link to https://issues.apache.org/jira/browse/FLINK-XXXX)

Released: <Flink Version>

Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).

Motivation

With FLIP-143 we introduced the unified Sink API to make it easier for connector developers to support batch and streaming scenarios. While developing the unified Sink API it was already noted that the unified Sink API might not be flexible enough to support all scenarios from the beginning.

With this document, we mainly focus on the small-file-compaction problem which is one of the not yet covered scenarios. The problem manifests when using the FileSink  and you want to guarantee a certain file size and not end up with a lot of small files (i.e. one file per parallel subtask). It becomes very important for modern data lake use cases where usually the data is written in some kind of columnar format (Parquet or ORC) and larger files are recommended to amplify the read performance. A few examples are the Iceberg sink [1], Delta lake [2], or Hive. For Hive in particular we already have a sink that supports compaction but it is not generally applicable and only available in Flink’s Table API [3].

The goal of this document is to extend the unified Sink API to broaden the spectrum of supported scenarios and fix the small-file-compaction  problem.

Alternative 1 Global Sink Coordinator:

The idea is similar to the SourceCoordinator which was implemented as part of FLIP-27 for the unified Source interface. On the contrary, to the unified source interface the Sink coordinator is optional and not all sinks must implement it.

The global sink coordinator solves the small-file-compaction problem by combining committables (files with sizing information) together until a certain threshold is reached. Afterward the combined committable is forwarded to the committers that do the actual file merging and write it to the final location.

Proposed Changes

Extend the unified Sink interface by allowing to create sink coordinator similar to the `SourceEnumerator` [4]. The creation is optional so the forwarding of the committable to the Job Manager will also only happen if the connector specifies one. In general, the behaviour of sending the committables does not change (sending them on preSnapshot) and the sink coordinator needs to write them into the checkpoint.

Limitations

Unfortunately, checkpoint for the sink coordinator is not easily possible because the coordinators are checkpointed before the operators in the pipeline. In the case of the sources, this works well because the source coordinator is basically the first operator of the pipeline. For sinks, it means that the sink coordinator would be snapshotting its state before the sink writers leading to potential data loss. Making the sink coordinator stateful would require first changes to the coordinator framework around how they are doing checkpoints.

Another limitation is that committables to the sink coordinators are sent via RPC calls which restrict the payload size. If the committables contain data this will probably not be possible anymore with this approach.

Alternative 2 Committable Aggregator Operator:

This approach is comparable to the current implementation in Flink’s Table API to solve the small-file-compaction problem [5] but we make it more extensible.

The committable aggregators are optional operators between the sink writers and committers. The user can define the key for the exchange to the comittable aggregator, the parallelism of the committable aggregator, and the key for exchange from the committable aggregators to the committers.

Proposed Changes

We propose to extend the unified Sink interface with the following changes. The changes to the unified sink should be backward compatible because only a new method with a default implementation is added to the sink interface.


Sink.java
    /**
     * Creates a {@link CommittableAggregator} that can aggregate committables before they are
     * finally committed.
     */
    default Optional<CommittableAggregator<CommT>> createCommittableAggregator() {
        return Optional.empty();
    }
CommittableAggregator
public interface CommittableAggregator<CommT> {

    /**
     * Computes a key for a committable to determine the distribution from sink writers across downstream aggregators.
     *
     * @return the key derived from the committable
     */
    default <KEY> Optional<KeySelector<CommT, KEY>> getPreAggregateSelector() {
        return Optional.empty();
    }

    /**
     * Computes a key for a committable to determine the distribution across downstream committers.
     *
     * @return the key derived from the committable
     */
    default <KEY> Optional<KeySelector<CommT, KEY>> getPostAggregateSelector() {
        return Optional.empty();
    }

    /**
     * Aggregates the committables emitted by the {@link SinkWriter}s and emits a new set of
     * committables to the downstream committers.
     *
     * @param committables committables sent by the {@link SinkWriter}s.
     * @return committables forwarded to the committers
     */
    List<CommT> aggregate(List<CommT> committables);
}

In the case of the small-file-compaction problem, the parallelism will likely be set to 1 for the committable aggregator so that compaction across multiple subtasks is possible. In addition, the approach also supports compaction across multiple checkpoints if the committables in on checkpoint do not meet the required size.

A first example POC can be found here.

Limitations

Although the parallelism and the exchanges between the committable aggregator and the sink writer/committer are configurable new use cases for the unified sink might require more flexibility. In the future, we may want to add a new operator supporting a new use case. Over time this can lead to a fragmented interface with a lot of different options which may or may not be compatible.

Alternative 3 Custom sink topology:

In general, the most important part of a sink is how the commit mechanism works because usually data which is already committed is hard to revert and may lead to problems on the consumer side. The custom sink topology isolated the committer and global committer as the only components which the Sink offers and the former sink writers are only syntactic sugar. Connector developers can implement their own custom topologies which generate committables and are passed to the committers.

By providing the full flexibility of building the topology which is executed as part of the sink it should also be easy to build a topology to support small file compaction.

Proposed Changes

The plan will be to expose the DataStream incoming to the sink directly by extending the Sink interface to create a custom topology. The default is still using the sink writer topology to not break existing sinks.

Sink.java
 @Override
 default Topology<InputT> createTopology() {
        return new WriterOnlyTopology<>(this);
    }

During the instantiation of the DataStreamSink, the correct topology is created by configuring a different translator. The DataStreamTopology gives connector developers direct access to the incoming datastream and they only need to ensure that they return a stream with committables.

DataStreamSink
DataStreamSink.java
 protected DataStreamSink(DataStream<T> inputStream, TopologySink<T, ?, ?> sink) {
        Topology<T> topology = sink.createTopology();
        TopologyTranslator translator = TopologyTranslators.getTranslator(topology.getClass());

        try {
            DataStream<byte[]> committableStream = translator.translate(inputStream, topology);

            // ensures the properties are set to the first transformation...
            transformation = findFirstTransformation(inputStream, committableStream);

            SinkCommitterTransformation<?, ?> sinkCommitterTransformation =
                    new SinkCommitterTransformation<>(
                            committableStream.getTransformation(), sink, "committer");
            inputStream.getExecutionEnvironment().addOperator(sinkCommitterTransformation);
        } catch (Exception exception) {
            throw new RuntimeException(exception);
        }
    }
    
  private static class TopologyTranslators {
        private static final Map<Class<?>, TopologyTranslator> TRANSLATOR_MAP =
                new HashMap<Class<?>, TopologyTranslator>() {
                    {
                        this.put(WriterOnlyTopology.class, new WriterOnlyTopologyTranslator());
                        this.put(DataStreamTopology.class, new DataStreamTopologyTranslator());
                    }
                };

        public static TopologyTranslator getTranslator(Class<?> clazz) {
            return TRANSLATOR_MAP.get(clazz);
        }
    }

Limitations

This approach probably allows implementing all sink scenarios because of its high flexibility. Unfortunately, it also goes a step back considering the batch and stream unification. If users start implementing stateful topologies they need to be familiar with both concepts to fully support them.

Another problem is that currently, flink-core does not know the DataStream classes currently which are in flink-streaming-java  to circumvent this issue we could have the implementation in flink-streaming-java and only provide some base interface in flink-core. This has a downside in that it adds unnecessary complexity and might confuse users.

Rejected Alternatives

TBD


References

[1] https://github.com/apache/iceberg/pull/3213

[2] https://docs.delta.io/latest/best-practices.html#compact-files

[3] https://docs.google.com/document/d/1cdlyoqgBq9yJEiHFBziimIoKHapQiEY2-0Tn8IF6G-c/edit#heading=h.nebq49xwdxns

[4] https://github.com/apache/flink/blob/003df215b482c246c48c147b63b56608c6557cba/flink-core/src/main/java/org/apache/flink/api/connector/source/Source.java#L63

[5] https://github.com/apache/flink/blob/cebf1bd9a46447d32dcc173b0105c4e4390cdbb6/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/stream/compact/CompactMessages.java#L32




  • No labels