Skip to content

Commit

Permalink
[HUDI-7266] Add clustering metric for flink (apache#10420)
Browse files Browse the repository at this point in the history
  • Loading branch information
LXin96 authored Jan 7, 2024
1 parent 397a2fd commit 478833a
Show file tree
Hide file tree
Showing 5 changed files with 158 additions and 1 deletion.
Original file line number Diff line number Diff line change
@@ -0,0 +1,105 @@
/*
* 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.hudi.metrics;

import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieInstantTimeGenerator;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.sink.clustering.ClusteringOperator;
import org.apache.hudi.sink.clustering.ClusteringPlanOperator;

import org.apache.flink.metrics.MetricGroup;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.text.ParseException;
import java.time.Duration;
import java.time.Instant;

/**
* Metrics for flink clustering.
*/
public class FlinkClusteringMetrics extends FlinkWriteMetrics {

private static final Logger LOG = LoggerFactory.getLogger(FlinkClusteringMetrics.class);

/**
* Key for clustering timer.
*/
private static final String CLUSTERING_KEY = "clustering";

/**
* Number of pending clustering instants.
*
* @see ClusteringPlanOperator
*/
private long pendingClusteringCount;

/**
* Duration between the earliest pending clustering instant time and now in seconds.
*
* @see ClusteringPlanOperator
*/
private long clusteringDelay;

/**
* Cost for consuming a clustering operation in milliseconds.
*
* @see ClusteringOperator
*/
private long clusteringCost;

public FlinkClusteringMetrics(MetricGroup metricGroup) {
super(metricGroup, CLUSTERING_KEY);
}

@Override
public void registerMetrics() {
super.registerMetrics();
metricGroup.gauge(getMetricsName(actionType, "pendingClusteringCount"), () -> pendingClusteringCount);
metricGroup.gauge(getMetricsName(actionType, "clusteringDelay"), () -> clusteringDelay);
metricGroup.gauge(getMetricsName(actionType, "clusteringCost"), () -> clusteringCost);
}

public void setPendingClusteringCount(long pendingClusteringCount) {
this.pendingClusteringCount = pendingClusteringCount;
}

public void setFirstPendingClusteringInstant(Option<HoodieInstant> firstPendingClusteringInstant) {
try {
if (!firstPendingClusteringInstant.isPresent()) {
this.clusteringDelay = 0L;
} else {
Instant start = HoodieInstantTimeGenerator.parseDateFromInstantTime((firstPendingClusteringInstant.get()).getTimestamp()).toInstant();
this.clusteringDelay = Duration.between(start, Instant.now()).getSeconds();
}
} catch (ParseException e) {
LOG.warn("Invalid input clustering instant" + firstPendingClusteringInstant);
}
}

public void startClustering() {
startTimer(CLUSTERING_KEY);
}

public void endClustering() {
this.clusteringCost = stopTimer(CLUSTERING_KEY);
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -35,13 +35,15 @@
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.exception.HoodieClusteringException;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.metrics.FlinkClusteringMetrics;
import org.apache.hudi.sink.CleanFunction;
import org.apache.hudi.table.HoodieFlinkTable;
import org.apache.hudi.table.action.HoodieWriteMetadata;
import org.apache.hudi.util.ClusteringUtil;
import org.apache.hudi.util.FlinkWriteClients;

import org.apache.flink.configuration.Configuration;
import org.apache.flink.metrics.MetricGroup;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -88,6 +90,8 @@ public class ClusteringCommitSink extends CleanFunction<ClusteringCommitEvent> {
*/
private transient Map<String, HoodieClusteringPlan> clusteringPlanCache;

private transient FlinkClusteringMetrics clusteringMetrics;

public ClusteringCommitSink(Configuration conf) {
super(conf);
this.conf = conf;
Expand All @@ -102,6 +106,7 @@ public void open(Configuration parameters) throws Exception {
this.commitBuffer = new HashMap<>();
this.clusteringPlanCache = new HashMap<>();
this.table = writeClient.getHoodieTable();
registerMetrics();
}

@Override
Expand Down Expand Up @@ -194,6 +199,7 @@ private void doCommit(String instant, HoodieClusteringPlan clusteringPlan, Colle
this.writeClient.completeTableService(
TableServiceType.CLUSTER, writeMetadata.getCommitMetadata().get(), table, instant, Option.of(HoodieListData.lazy(writeMetadata.getWriteStatuses())));

clusteringMetrics.updateCommitMetrics(instant, writeMetadata.getCommitMetadata().get());
// whether to clean up the input base parquet files used for clustering
if (!conf.getBoolean(FlinkOptions.CLEAN_ASYNC_ENABLED) && !isCleaning) {
LOG.info("Running inline clean");
Expand Down Expand Up @@ -229,4 +235,10 @@ private static Map<String, List<String>> getPartitionToReplacedFileIds(
.filter(fg -> !newFilesWritten.contains(fg))
.collect(Collectors.groupingBy(HoodieFileGroupId::getPartitionPath, Collectors.mapping(HoodieFileGroupId::getFileId, Collectors.toList())));
}

private void registerMetrics() {
MetricGroup metrics = getRuntimeContext().getMetricGroup();
clusteringMetrics = new FlinkClusteringMetrics(metrics);
clusteringMetrics.registerMetrics();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@
import org.apache.hudi.io.storage.HoodieAvroFileReader;
import org.apache.hudi.io.storage.HoodieFileReader;
import org.apache.hudi.io.storage.HoodieFileReaderFactory;
import org.apache.hudi.metrics.FlinkClusteringMetrics;
import org.apache.hudi.sink.bulk.BulkInsertWriterHelper;
import org.apache.hudi.sink.bulk.sort.SortOperatorGen;
import org.apache.hudi.sink.utils.NonThrownExecutor;
Expand All @@ -58,6 +59,7 @@
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.metrics.Gauge;
import org.apache.flink.metrics.MetricGroup;
import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.streaming.api.graph.StreamConfig;
import org.apache.flink.streaming.api.operators.BoundedOneInput;
Expand Down Expand Up @@ -127,6 +129,8 @@ public class ClusteringOperator extends TableStreamOperator<ClusteringCommitEven
*/
private transient NonThrownExecutor executor;

private transient FlinkClusteringMetrics clusteringMetrics;

public ClusteringOperator(Configuration conf, RowType rowType) {
// copy a conf let following modification not to impact the global conf
this.conf = new Configuration(conf);
Expand Down Expand Up @@ -170,6 +174,8 @@ public void open() throws Exception {
}

this.collector = new StreamRecordCollector<>(output);

registerMetrics();
}

@Override
Expand Down Expand Up @@ -213,6 +219,7 @@ public void endInput() {
// -------------------------------------------------------------------------

private void doClustering(String instantTime, List<ClusteringOperation> clusteringOperations) throws Exception {
clusteringMetrics.startClustering();
BulkInsertWriterHelper writerHelper = new BulkInsertWriterHelper(this.conf, this.table, this.writeConfig,
instantTime, this.taskID, getRuntimeContext().getNumberOfParallelSubtasks(), getRuntimeContext().getAttemptNumber(),
this.rowType, true);
Expand Down Expand Up @@ -247,6 +254,7 @@ instantTime, this.taskID, getRuntimeContext().getNumberOfParallelSubtasks(), get
}

List<WriteStatus> writeStatuses = writerHelper.getWriteStatuses(this.taskID);
clusteringMetrics.endClustering();
collector.collect(new ClusteringCommitEvent(instantTime, getFileIds(clusteringOperations), writeStatuses, this.taskID));
writerHelper.close();
}
Expand Down Expand Up @@ -388,4 +396,10 @@ public void setExecutor(NonThrownExecutor executor) {
public void setOutput(Output<StreamRecord<ClusteringCommitEvent>> output) {
this.output = output;
}

private void registerMetrics() {
MetricGroup metrics = getRuntimeContext().getMetricGroup();
clusteringMetrics = new FlinkClusteringMetrics(metrics);
clusteringMetrics.registerMetrics();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -26,18 +26,22 @@
import org.apache.hudi.common.util.ClusteringUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.metrics.FlinkClusteringMetrics;
import org.apache.hudi.table.HoodieFlinkTable;
import org.apache.hudi.util.ClusteringUtil;
import org.apache.hudi.util.FlinkTables;
import org.apache.hudi.util.FlinkWriteClients;

import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.metrics.MetricGroup;
import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.operators.Output;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;

import java.util.List;

/**
* Operator that generates the clustering plan with pluggable strategies on finished checkpoints.
*
Expand All @@ -57,6 +61,8 @@ public class ClusteringPlanOperator extends AbstractStreamOperator<ClusteringPla
@SuppressWarnings("rawtypes")
private transient HoodieFlinkTable table;

private transient FlinkClusteringMetrics clusteringMetrics;

public ClusteringPlanOperator(Configuration conf) {
this.conf = conf;
}
Expand All @@ -65,6 +71,7 @@ public ClusteringPlanOperator(Configuration conf) {
public void open() throws Exception {
super.open();
this.table = FlinkTables.createTable(conf, getRuntimeContext());
registerMetrics();
// when starting up, rolls back all the inflight clustering instants if there exists,
// these instants are in priority for scheduling task because the clustering instants are
// scheduled from earliest(FIFO sequence).
Expand All @@ -88,10 +95,17 @@ public void notifyCheckpointComplete(long checkpointId) {
}

private void scheduleClustering(HoodieFlinkTable<?> table, long checkpointId) {
List<HoodieInstant> pendingClusteringInstantTimes =
ClusteringUtils.getPendingClusteringInstantTimes(table.getMetaClient());
// the first instant takes the highest priority.
Option<HoodieInstant> firstRequested = Option.fromJavaOptional(
ClusteringUtils.getPendingClusteringInstantTimes(table.getMetaClient()).stream()
pendingClusteringInstantTimes.stream()
.filter(instant -> instant.getState() == HoodieInstant.State.REQUESTED).findFirst());

// record metrics
clusteringMetrics.setFirstPendingClusteringInstant(firstRequested);
clusteringMetrics.setPendingClusteringCount(pendingClusteringInstantTimes.size());

if (!firstRequested.isPresent()) {
// do nothing.
LOG.info("No clustering plan for checkpoint " + checkpointId);
Expand Down Expand Up @@ -136,4 +150,10 @@ private void scheduleClustering(HoodieFlinkTable<?> table, long checkpointId) {
public void setOutput(Output<StreamRecord<ClusteringPlanEvent>> output) {
this.output = output;
}

private void registerMetrics() {
MetricGroup metrics = getRuntimeContext().getMetricGroup();
clusteringMetrics = new FlinkClusteringMetrics(metrics);
clusteringMetrics.registerMetrics();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,10 @@ public class ClusteringFunctionWrapper {
* Function that generates the {@code HoodieClusteringPlan}.
*/
private ClusteringPlanOperator clusteringPlanOperator;
/**
* Output to collect the clustering plan events.
*/
private CollectorOutput<ClusteringPlanEvent> planEventOutput;
/**
* Output to collect the clustering commit events.
*/
Expand Down Expand Up @@ -83,6 +87,8 @@ public ClusteringFunctionWrapper(Configuration conf, StreamTask<?, ?> streamTask

public void openFunction() throws Exception {
clusteringPlanOperator = new ClusteringPlanOperator(conf);
planEventOutput = new CollectorOutput<>();
clusteringPlanOperator.setup(streamTask, streamConfig, planEventOutput);
clusteringPlanOperator.open();

clusteringOperator = new ClusteringOperator(conf, TestConfigurations.ROW_TYPE);
Expand Down

0 comments on commit 478833a

Please sign in to comment.