-
Notifications
You must be signed in to change notification settings - Fork 2.3k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Flink:backport PR to 1.15 #7360: Implement data statistics coordinato…
…r to aggregate data statistics from operator subtasks (#8749) Co-authored-by: gang_ye <[email protected]>
- Loading branch information
1 parent
3e522e8
commit 6d3b0b7
Showing
13 changed files
with
1,329 additions
and
76 deletions.
There are no files selected for viewing
71 changes: 71 additions & 0 deletions
71
...v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,71 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, | ||
* software distributed under the License is distributed on an | ||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
* KIND, either express or implied. See the License for the | ||
* specific language governing permissions and limitations | ||
* under the License. | ||
*/ | ||
package org.apache.iceberg.flink.sink.shuffle; | ||
|
||
import java.io.Serializable; | ||
import org.apache.flink.api.common.typeutils.TypeSerializer; | ||
import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; | ||
import org.apache.iceberg.relocated.com.google.common.base.Preconditions; | ||
|
||
/** | ||
* AggregatedStatistics is used by {@link DataStatisticsCoordinator} to collect {@link | ||
* DataStatistics} from {@link DataStatisticsOperator} subtasks for specific checkpoint. It stores | ||
* the merged {@link DataStatistics} result from all reported subtasks. | ||
*/ | ||
class AggregatedStatistics<D extends DataStatistics<D, S>, S> implements Serializable { | ||
|
||
private final long checkpointId; | ||
private final DataStatistics<D, S> dataStatistics; | ||
|
||
AggregatedStatistics(long checkpoint, TypeSerializer<DataStatistics<D, S>> statisticsSerializer) { | ||
this.checkpointId = checkpoint; | ||
this.dataStatistics = statisticsSerializer.createInstance(); | ||
} | ||
|
||
AggregatedStatistics(long checkpoint, DataStatistics<D, S> dataStatistics) { | ||
this.checkpointId = checkpoint; | ||
this.dataStatistics = dataStatistics; | ||
} | ||
|
||
long checkpointId() { | ||
return checkpointId; | ||
} | ||
|
||
DataStatistics<D, S> dataStatistics() { | ||
return dataStatistics; | ||
} | ||
|
||
void mergeDataStatistic(String operatorName, long eventCheckpointId, D eventDataStatistics) { | ||
Preconditions.checkArgument( | ||
checkpointId == eventCheckpointId, | ||
"Received unexpected event from operator %s checkpoint %s. Expected checkpoint %s", | ||
operatorName, | ||
eventCheckpointId, | ||
checkpointId); | ||
dataStatistics.merge(eventDataStatistics); | ||
} | ||
|
||
@Override | ||
public String toString() { | ||
return MoreObjects.toStringHelper(this) | ||
.add("checkpointId", checkpointId) | ||
.add("dataStatistics", dataStatistics) | ||
.toString(); | ||
} | ||
} |
133 changes: 133 additions & 0 deletions
133
...link/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,133 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, | ||
* software distributed under the License is distributed on an | ||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
* KIND, either express or implied. See the License for the | ||
* specific language governing permissions and limitations | ||
* under the License. | ||
*/ | ||
package org.apache.iceberg.flink.sink.shuffle; | ||
|
||
import java.util.Set; | ||
import org.apache.flink.api.common.typeutils.TypeSerializer; | ||
import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; | ||
import org.apache.iceberg.relocated.com.google.common.collect.Sets; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
/** | ||
* AggregatedStatisticsTracker is used by {@link DataStatisticsCoordinator} to track the in progress | ||
* {@link AggregatedStatistics} received from {@link DataStatisticsOperator} subtasks for specific | ||
* checkpoint. | ||
*/ | ||
class AggregatedStatisticsTracker<D extends DataStatistics<D, S>, S> { | ||
private static final Logger LOG = LoggerFactory.getLogger(AggregatedStatisticsTracker.class); | ||
private static final double ACCEPT_PARTIAL_AGGR_THRESHOLD = 90; | ||
private final String operatorName; | ||
private final TypeSerializer<DataStatistics<D, S>> statisticsSerializer; | ||
private final int parallelism; | ||
private final Set<Integer> inProgressSubtaskSet; | ||
private volatile AggregatedStatistics<D, S> inProgressStatistics; | ||
|
||
AggregatedStatisticsTracker( | ||
String operatorName, | ||
TypeSerializer<DataStatistics<D, S>> statisticsSerializer, | ||
int parallelism) { | ||
this.operatorName = operatorName; | ||
this.statisticsSerializer = statisticsSerializer; | ||
this.parallelism = parallelism; | ||
this.inProgressSubtaskSet = Sets.newHashSet(); | ||
} | ||
|
||
AggregatedStatistics<D, S> updateAndCheckCompletion( | ||
int subtask, DataStatisticsEvent<D, S> event) { | ||
long checkpointId = event.checkpointId(); | ||
|
||
if (inProgressStatistics != null && inProgressStatistics.checkpointId() > checkpointId) { | ||
LOG.info( | ||
"Expect data statistics for operator {} checkpoint {}, but receive event from older checkpoint {}. Ignore it.", | ||
operatorName, | ||
inProgressStatistics.checkpointId(), | ||
checkpointId); | ||
return null; | ||
} | ||
|
||
AggregatedStatistics<D, S> completedStatistics = null; | ||
if (inProgressStatistics != null && inProgressStatistics.checkpointId() < checkpointId) { | ||
if ((double) inProgressSubtaskSet.size() / parallelism * 100 | ||
>= ACCEPT_PARTIAL_AGGR_THRESHOLD) { | ||
completedStatistics = inProgressStatistics; | ||
LOG.info( | ||
"Received data statistics from {} subtasks out of total {} for operator {} at checkpoint {}. " | ||
+ "Complete data statistics aggregation at checkpoint {} as it is more than the threshold of {} percentage", | ||
inProgressSubtaskSet.size(), | ||
parallelism, | ||
operatorName, | ||
checkpointId, | ||
inProgressStatistics.checkpointId(), | ||
ACCEPT_PARTIAL_AGGR_THRESHOLD); | ||
} else { | ||
LOG.info( | ||
"Received data statistics from {} subtasks out of total {} for operator {} at checkpoint {}. " | ||
+ "Aborting the incomplete aggregation for checkpoint {}", | ||
inProgressSubtaskSet.size(), | ||
parallelism, | ||
operatorName, | ||
checkpointId, | ||
inProgressStatistics.checkpointId()); | ||
} | ||
|
||
inProgressStatistics = null; | ||
inProgressSubtaskSet.clear(); | ||
} | ||
|
||
if (inProgressStatistics == null) { | ||
LOG.info("Starting a new data statistics for checkpoint {}", checkpointId); | ||
inProgressStatistics = new AggregatedStatistics<>(checkpointId, statisticsSerializer); | ||
inProgressSubtaskSet.clear(); | ||
} | ||
|
||
if (!inProgressSubtaskSet.add(subtask)) { | ||
LOG.debug( | ||
"Ignore duplicated data statistics from operator {} subtask {} for checkpoint {}.", | ||
operatorName, | ||
subtask, | ||
checkpointId); | ||
} else { | ||
inProgressStatistics.mergeDataStatistic( | ||
operatorName, | ||
event.checkpointId(), | ||
DataStatisticsUtil.deserializeDataStatistics( | ||
event.statisticsBytes(), statisticsSerializer)); | ||
} | ||
|
||
if (inProgressSubtaskSet.size() == parallelism) { | ||
completedStatistics = inProgressStatistics; | ||
LOG.info( | ||
"Received data statistics from all {} operators {} for checkpoint {}. Return last completed aggregator {}.", | ||
parallelism, | ||
operatorName, | ||
inProgressStatistics.checkpointId(), | ||
completedStatistics.dataStatistics()); | ||
inProgressStatistics = new AggregatedStatistics<>(checkpointId + 1, statisticsSerializer); | ||
inProgressSubtaskSet.clear(); | ||
} | ||
|
||
return completedStatistics; | ||
} | ||
|
||
@VisibleForTesting | ||
AggregatedStatistics<D, S> inProgressStatistics() { | ||
return inProgressStatistics; | ||
} | ||
} |
Oops, something went wrong.