Skip to content

Conversation

@yegangy0718
Copy link
Contributor

@yegangy0718 yegangy0718 commented Dec 8, 2022

This PR is created as part of issue #6303 and project https://github.com/apache/iceberg/projects/27

In this PR, we focus on creating the ShuffleOperator for bin packing based on traffic distribution statistics. (the second one in the issue #6303).

Changes:

  1. Implement ShuffleOperator which will be added before Iceberg Writer operator to collect data distribution based on key(generated from provided KeySelector)
  2. Implement ShuffleRecordWrapper which contains either the record or data distribution information

I will have following up PRs to implement ShuffleCoordinator, the data distribution sending and receiving logic between coordinator and operator, and etc.

@github-actions github-actions bot added the flink label Dec 8, 2022
@yegangy0718 yegangy0718 force-pushed the 20221206-oss-shuffle-operator branch from 65973d0 to aaecd1d Compare December 8, 2022 06:05
@Override
public void initializeState(StateInitializationContext context) throws Exception {
localDataStatisticsMap = Maps.newHashMap();
globalDataDistributionWeightState =
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The global data distribution should be the same (or samish) for every instance of the operator.
Shall we use getBroadcastState instead?
Based on my understanding that would better approximate our requirements for redistributing the state when rescaling. Even if there are some small discrepancies in the values between the different operation instances, we should not be concerned too much about them.

WDYT?

Copy link
Contributor

@stevenzwu stevenzwu Dec 9, 2022

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@pvary discussed this with me offline a few days ago. Flink doc said that broadcast is usually used with broadcast stream. But it may just work for regular stream. This is the part yet to be verified. @yegangy0718 can you experiment the broadcast with some unit test (especially for rescale)?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

OK. Let me take some time to investigate if BroadcastState can be applied to this case.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

My current understanding is, when using broadcast, there will be two streams. One stream will be send to all tasks, and the other stream won’t be send to all tasks. Then connecting them together by calling connect . The task can use getBroadcastState to get the broadcast stream.
From concept perspective, yes, it’s similar to what we are dong. Global data statistics will be the broadcast one.
But for our case, there is no stream/place publish the broadcast state, where the shuffle operator can get BroadcastState? Do you mean the shuffle operator publishes the data statistics once receiving from coordinator and then in function initializeState shuffle operator can get BroadcastState?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's merge this change, and we can play around the Broadcaststate in a followup PR. WTYT?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

SGTM. I can create a followup PR to add scaling up test case to check the behavior for BroadcastState.

}

@Test
public void testInitializeState() throws Exception {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Might as well add tests for the rebalance case if it is not too complicated

Copy link
Contributor Author

@yegangy0718 yegangy0718 Jan 4, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The logic to send data statistics from operator to coordinator and data statistics aggregation function in coordinator are not defined in the PR. Thus the global state is always empty when restoring.
I will add the rebalance case in the next PR once the above parts are there.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

not sure if this test is necessary. seems too trivial to test.

MockEnvironment env = new MockEnvironmentBuilder().build();
AbstractStateBackend abstractStateBackend = new HashMapStateBackend();
CloseableRegistry cancelStreamRegistry = new CloseableRegistry();
return abstractStateBackend.createOperatorStateBackend(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe put some data here, and check that the state is restored as expected?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@yegangy0718 maybe we should use OneInputStreamOperatorTestHarness. you can refer to TestIcebergStreamWriter.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We get globalDataStatistics from coordinator and then save the value during snapshot. The function handleOperatorEvent to receive globalDataStatistics from coordinator is not implemented in this PR yet. Thus we won't be able to test that part.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I will add a test using OneInputStreamOperatorTestHarness in latest commit. After implementing the globalDataStatistics setting logic in next PR, I will add the test to take snapshot and restore from new state to verify state is restored as expected.


private static final long serialVersionUID = 1L;

private final Map<K, Long> globalDataDistributionWeight;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

wasn't this simply channel Id in the design spec ?

Copy link
Contributor Author

@yegangy0718 yegangy0718 Dec 13, 2022

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We plan to create a StreamPartitioner which is chained with the shuffle operator to takes the global distribution via ShuffleRecordWrapper, and then generate the channel assignment at there

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@zinking what do you mean channel Id? the downstream channel/subtask id?

*/
class DataStatisticsFactory<K> {

DataStatistics<K> createDataStatistics() {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I am not sure if there is much value of this factory class. even if we want to keep it, this method should be called createMapStatistics.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I would feel that we either need a clean method to the DataStatistics API, or need to abstract away the way to recreate a new one after sending it to the JobManager (at checkpoint). I do not feel that the Operator should know which type of statistics is used.

Alternatively we could use statistics which automatically keep historical data - but I would not bother with them in this phase

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The factory will create different types DataStatistics based on different scenarios like MapStatistics for low-cardinary and SketchStatistics/ DigestStatistics for high-cardinary. Thus I would prefer to keep it general.
Regarding whether we need this factory, it depends. We can either use ShuffleOperatorFactory to get config(low or high cardinary or range mode) and then pass the right data statistics to operator and coordinator, or use this factory, pass it into operator and coordinator to create DataStatistics by calling this function.

public void merge(DataStatistics<K> other) {
Preconditions.checkArgument(
other instanceof MapDataStatistics, "Can not merge this type of statistics: " + other);
MapDataStatistics<K> mapDataStatistic = (MapDataStatistics<K>) other;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: I would call this otherStatistics

}

@VisibleForTesting
ListStateDescriptor<DataStatistics<K>> generateGlobalDataDistributionWeightDescriptor() {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

not sure if this method is necessary for code reuse. not sure why it is necessary to check it in testing.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

we use it in unit test now to get the state to make sure state is not null like what SourceOperatorTest did https://github.com/apache/flink/blob/68b37fb867374df5a201f0b170e35c21266e5d7b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/SourceOperatorTest.java#L87
I was thinking we can use it to verify globalStatisticsState was updated as expected after taking snapshot.
But actually, we can also create a test-only visible function to return globalStatisticsState directly if we need to check its value.


private final KeySelector<T, K> keySelector;
private final OperatorEventGateway operatorEventGateway;
// key is generated by applying KeySelector to record
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: these comments seem not relevant anymore

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

removed

// key is generated by applying KeySelector to record
// value is the times key occurs
// TODO: support to store statistics for high cardinality cases
private transient DataStatistics<K> localDataStatistics;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: simplify the names a little by removing Data

@Override
public void initializeState(StateInitializationContext context) throws Exception {
localDataStatistics = dataStatisticsFactory.createDataStatistics();
globalDataStatisticsState =
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

we need to check if context is restored

if (context.isRestored()) {

Copy link
Contributor Author

@yegangy0718 yegangy0718 Jan 10, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

will add the check to line 81

    if (context.isRestored() && globalStatisticsState.get() != null
        && globalStatisticsState.get().iterator().hasNext())

since we still need to initialize the globalDataStatisticsState variable.
If we set globalDataStatisticsState to null when context.isRestored() = false, then when taking snapshot, we cannot execute globalStatisticsState.add(globalStatistics)

@yegangy0718 yegangy0718 force-pushed the 20221206-oss-shuffle-operator branch from 7330fce to 4c19c1a Compare January 18, 2023 02:03
@yegangy0718
Copy link
Contributor Author

Hi @stevenzwu if we decide to keep shuffling implementation in the Iceberg repo, could you help to take another look at the PR? Thanks!

* shuffle record to improve data clustering while maintaining relative balanced traffic
* distribution to downstream subtasks.
*/
class ShuffleOperator<T, K> extends AbstractStreamOperator<ShuffleRecordWrapper<T, K>>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I have been thinking about the name of this class. This operator technically only does statistics calculation. Hence ShuffleOperator sounds like a misleading name. But StatisticsOperator is too generic. Maybe DataStatisticsOperator?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

DataStatisticsOperator is better than StatisticsOperator. I'm OK to update the class name since this operator indeed only collects data statistics. I will rename ShuffleRecordWrapper to DataStatisticsAndRecordWrapper as well.


// keySelector will be used to generate key from data for collecting data statistics
private final KeySelector<T, K> keySelector;
private final OperatorEventGateway operatorEventGateway;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it doesn't seem to be in use.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it will be used in the next PR with the shuffle coordinator to aggregate statistics globally

* shuffle record to improve data clustering while maintaining relative balanced traffic
* distribution to downstream subtasks.
*/
class DataStatisticsOperator<T, K> extends AbstractStreamOperator<DataStatisticsAndRecordWrapper<T, K>>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

a quick question: Do we need to override finish() to cover the bounded case?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I will be fine either way. this streaming operator is mainly responsible for calculating and reporting statistics periodically. with finish(), it is probably not important to report the last statistics.

Note that for real batch jobs, the shuffling will be different. Statistics are probably sampled and calculated based on all the input data. it is a different shuffling compared to stream shuffling.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ok, I see. Thank you for your explanation.

@Override
public void add(K key) {
// increase count of occurrence by one in the dataStatistics map
statistics.put(key, statistics.getOrDefault(key, 0L) + 1L);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit

Suggested change
statistics.put(key, statistics.getOrDefault(key, 0L) + 1L);
statistics.merge(key, 1L, Long::sum);

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

will update in latest commit

Preconditions.checkArgument(
otherStatistics instanceof MapDataStatistics,
"Can not merge this type of statistics: " + otherStatistics);
MapDataStatistics<K> mapDataStatistic = (MapDataStatistics<K>) otherStatistics;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: if the input is not of type MapDataStatistics during the type conversion in line 48, will throw an error on its own. Therefore, whether it is necessary to perform this checkArgument?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think if the input is not in the right type, it would be better to throw IllegalArgumentException.
I checked the other places as well, for example at

Preconditions.checkArgument(term instanceof UnboundTerm, "Term must be unbound");

and
Preconditions.checkArgument(flinkType instanceof RowType, "%s is not a RowType.", flinkType);

It's common that first checking type and then convert it into specific type.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I agree with @yegangy0718 that it is a little better to check the type IllegalArgumentException

otherStatistics instanceof MapDataStatistics,
"Can not merge this type of statistics: " + otherStatistics);
MapDataStatistics<K> mapDataStatistic = (MapDataStatistics<K>) otherStatistics;
mapDataStatistic.statistics.forEach(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

also use merge?

import org.junit.Before;
import org.junit.Test;

public class TestDataStatisticsOperator {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It seems that we haven't tested the recovery case?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yes, for now in the current DataStatisticsOperator implementation, we set globalStatisticsState in function DataStatisticsOperator#snapshotState. But to get globalStatisticsState, it needs to implement the function DataStatisticsOperator#handleOperatorEvent which we plan to do in another PR.

@hililiwei
Copy link
Contributor

hililiwei commented Mar 3, 2023

Left some comments, and I will think about them more deeply in the next few days. Thank you.

* shuffling, a filter and mapper are required to filter out the data distribution weight, unwrap the
* object and extract the original record type T.
*/
public class DataStatisticsAndRecordWrapper<T, K> implements Serializable {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@yegangy0718 ShuffleRecord probably still makes sense for this wrapper class.

I saw some other classes in Iceberg with names like FooAndBar. But they typically contains both Foo and Bar. Here is an or relationship. Hence I don't know if And is the most accurate name.

Regardless, I prefer us remove the Wrapper suffice from the name. it can be described in the Javadoc

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The reason I rename it is, in DataStatisticsOperator, it generates the object which contains either global data statistics or record, while the place where shuffle happens is at the partitioner. The name DataStatisticsAndRecord is closer to the usage of the class(transmit global data statistics).

Indeed, like you said, the other class which uses And contains both. For example, RecordAndPosition class, it contains both record and position.
How about naming it DataStatisticsOrRecord even though there is no such kind of or class in the current repo :(
WDYT

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

DataStatisticsOrRecord sounds good to me

@yegangy0718
Copy link
Contributor Author

Hi @hililiwei, thanks for reviewing the code change. I have handled and replied to your comments. Let me know if you have more thoughts.

* The wrapper class for data statistics and record. It is the only way for data statistics operator to send
* global data statistics to custom partitioner to distribute data based on statistics
*
* <p>DataStatisticsOrRecord is sent from {@link DataStatisticsOperator} to partitioner. It
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it seems that we can remove the comments before It contains either ....

We can also change It contains either ... to This wrapper class contains either ...

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I actually think it's good to keep DataStatisticsOrRecord is sent from {@link DataStatisticsOperator} to partitioner. since it tells the readers where the class/object is being used. Once readers get the context, then they understand the later part Once partitioner receives the data...

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

maybe add the additional context after the sentence of it contains either ....

Preconditions.checkArgument(
otherStatistics instanceof MapDataStatistics,
"Can not merge this type of statistics: " + otherStatistics);
MapDataStatistics<K> mapDataStatistic = (MapDataStatistics<K>) otherStatistics;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I agree with @yegangy0718 that it is a little better to check the type IllegalArgumentException

public void merge(DataStatistics<K> otherStatistics) {
Preconditions.checkArgument(
otherStatistics instanceof MapDataStatistics,
"Can not merge this type of statistics: " + otherStatistics);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: the error msg can be a little clearer. note that maybe we can add a String type() to the DataStatistics interface. we shouldn't dump the whole statistics from toString() in the error msg.

Map statistics can not merge with statistics type: " + otherStatistics.type()

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

how about getting the type for statistics from the class name
"Map statistics can not merge with " + otherStatistics.getClass()

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yes, simple class name works well too.


@Override
public void processElement(StreamRecord<T> streamRecord) throws Exception {
final K key = keySelector.getKey(streamRecord.getValue());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: Iceberg coding style doesn't use final for stack local variables.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

OK. Will remove final

public void processElement(StreamRecord<T> streamRecord) throws Exception {
final K key = keySelector.getKey(streamRecord.getValue());
localStatistics.add(key);
output.collect(new StreamRecord<>(DataStatisticsOrRecord.fromRecord(streamRecord.getValue())));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: maybe cache the value at the beginning of this method. T record = streamRecord.getValue(). I saw more of this style in the Iceberg code.

// an exact copy of globalStatistics
if (!globalStatistics.isEmpty() && getRuntimeContext().getIndexOfThisSubtask() == 0) {
globalStatisticsState.clear();
LOG.debug("Saving global statistics {} to state", globalStatistics);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: I think this should actually be info level logging. maybe add subtask index to the log.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

will update log level to info and add subtask id in latest commit

@Override
public void snapshotState(StateSnapshotContext context) throws Exception {
long checkpointId = context.getCheckpointId();
LOG.info("Taking data statistics operator snapshot for checkpoint {}", checkpointId);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: add subtask index to the log. E.g. Snapshotting data statistics operator for checkpoint {} in subtask {}

@stevenzwu stevenzwu changed the title Implement ShuffleOperator to collect data statistics Flink: Implement data statistics operator to collect traffic distribution for guiding smart shuffling Mar 28, 2023
@stevenzwu stevenzwu merged commit 47f42f5 into apache:master Mar 28, 2023
@stevenzwu
Copy link
Contributor

thanks @yegangy0718 for the contribution and @pvary and @hililiwei for the review

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Development

Successfully merging this pull request may close these issues.

5 participants