Skip to content

Commit

Permalink
update log message and comments
Browse files Browse the repository at this point in the history
  • Loading branch information
gang_ye committed Sep 22, 2023
1 parent 7bdb2b0 commit 02f3df1
Show file tree
Hide file tree
Showing 4 changed files with 142 additions and 6 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -185,7 +185,9 @@ private void sendDataStatisticsToSubtasks(

return null;
},
String.format("Failed to send global data statistics for checkpoint %d", checkpointId));
String.format(
"Failed to send operator %s coordinator global data statistics for checkpoint %d",
operatorName, checkpointId));
}

@Override
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,135 @@
/*
* 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.HashSet;
import java.util.Set;
import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/**
* GlobalStatisticsTracker is used by {@link DataStatisticsCoordinator} to track the in progress
* {@link GlobalStatistics} received from {@link DataStatisticsOperator} subtasks for specific
* checkpoint.
*/
@Internal
class GlobalStatisticsTracker<D extends DataStatistics<D, S>, S> {
private static final Logger LOG = LoggerFactory.getLogger(GlobalStatisticsTracker.class);
private static final double EXPECTED_DATA_STATISTICS_RECEIVED_PERCENTAGE = 90;
private final String operatorName;
private final TypeSerializer<DataStatistics<D, S>> statisticsSerializer;
private final int parallelism;
private volatile GlobalStatistics<D, S> inProgressStatistics;
private final Set<Integer> inProgressSubtaskSet;

GlobalStatisticsTracker(
String operatorName,
TypeSerializer<DataStatistics<D, S>> statisticsSerializer,
int parallelism) {
this.operatorName = operatorName;
this.statisticsSerializer = statisticsSerializer;
this.parallelism = parallelism;
this.inProgressSubtaskSet = new HashSet<>();
}

GlobalStatistics<D, S> receiveDataStatisticEventAndCheckCompletion(
int subtask, DataStatisticsEvent<D, S> event) {
long checkpointId = event.checkpointId();

if (inProgressStatistics != null && inProgressStatistics.checkpointId() > checkpointId) {
LOG.debug(
"Expect data statistics for operator {} checkpoint {}, but receive event from older checkpoint {}. Ignore it.",
operatorName,
inProgressStatistics.checkpointId(),
checkpointId);
return null;
}

GlobalStatistics<D, S> completedStatistics = null;
if (inProgressStatistics != null && inProgressStatistics.checkpointId() < checkpointId) {
if ((double) inProgressSubtaskSet.size() / parallelism * 100
>= EXPECTED_DATA_STATISTICS_RECEIVED_PERCENTAGE) {
completedStatistics = inProgressStatistics;
LOG.info(
"Received data statistics from {} subtasks out of total {} for operator {} at checkpoint {}. "
+ "Complete data statistics aggregation as it is more than the threshold of {} percentage",
inProgressSubtaskSet.size(),
parallelism,
operatorName,
inProgressStatistics.checkpointId(),
EXPECTED_DATA_STATISTICS_RECEIVED_PERCENTAGE);
} else {
LOG.info(
"Received data statistics from {} subtasks out of total {} for operator {} at checkpoint {}. "
+ "Aborting the incomplete aggregation for checkpoint {} "
+ "and starting a new data statistics for checkpoint {}",
inProgressSubtaskSet.size(),
parallelism,
operatorName,
inProgressStatistics.checkpointId(),
EXPECTED_DATA_STATISTICS_RECEIVED_PERCENTAGE,
checkpointId);
}

inProgressStatistics = null;
inProgressSubtaskSet.clear();
}

if (inProgressStatistics == null) {
inProgressStatistics = new GlobalStatistics<>(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 GlobalStatistics<>(checkpointId + 1, statisticsSerializer);
inProgressSubtaskSet.clear();
}

return completedStatistics;
}

@VisibleForTesting
GlobalStatistics<D, S> inProgressStatistics() {
return inProgressStatistics;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -74,7 +74,7 @@ public void receiveNewerDataStatisticEvent() {
aggregatedStatisticsTracker.receiveDataStatisticEventAndCheckCompletion(
0, checkpoint2Subtask0DataStatisticEvent));

// Checkpoint 2 is newer than checkpoint1, thus drop inProgressAggregator for checkpoint1
// Checkpoint 2 is newer than checkpoint1, thus dropping in progress statistics for checkpoint1
Assert.assertEquals(2, aggregatedStatisticsTracker.inProgressStatistics().checkpointId());
}

Expand All @@ -96,12 +96,11 @@ public void receiveOlderDataStatisticEventTest() {
DataStatisticsEvent<MapDataStatistics, Map<RowData, Long>>
checkpoint1Subtask1DataStatisticEvent =
DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer);
// Receive event from old checkpoint, aggregatedStatisticsAggregatorTracker won't return
// completed statistics and in progress statistics won't be updated
Assert.assertNull(
aggregatedStatisticsTracker.receiveDataStatisticEventAndCheckCompletion(
1, checkpoint1Subtask1DataStatisticEvent));
// Receive event from old checkpoint, aggregatedStatisticsAggregatorTracker
// lastCompletedAggregator
// and inProgressAggregator won't be updated
Assert.assertEquals(2, aggregatedStatisticsTracker.inProgressStatistics().checkpointId());
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -129,7 +129,7 @@ public void testCheckpointAndReset() throws Exception {
(DataStatisticsCoordinator<MapDataStatistics, Map<RowData, Long>>)
coordinator.getInternalCoordinator();
Assert.assertNotEquals(
"The restored shuffle coordinator should be a different instance",
"The restored coordinator should be a different instance",
restoredDataStatisticsCoordinator,
dataStatisticsCoordinator);
// Verify restored data statistics
Expand Down

0 comments on commit 02f3df1

Please sign in to comment.