Skip to content

Commit

Permalink
[flink] Introduce PartitionListener to enable custom actions in the F…
Browse files Browse the repository at this point in the history
…link committer for partitions. (#4398)
  • Loading branch information
Aitozi authored Oct 30, 2024
1 parent 3ed17b1 commit 0d064f9
Show file tree
Hide file tree
Showing 13 changed files with 636 additions and 32 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -86,6 +86,12 @@
<td>Duration</td>
<td>Set a time duration when a partition has no new data after this time duration, mark the done status to indicate that the data is ready.</td>
</tr>
<tr>
<td><h5>partition.idle-time-to-report-statistic</h5></td>
<td style="word-wrap: break-word;">1 h</td>
<td>Duration</td>
<td>Set a time duration when a partition has no new data after this time duration, start to report the partition statistics to hms.</td>
</tr>
<tr>
<td><h5>partition.time-interval</h5></td>
<td style="word-wrap: break-word;">(none)</td>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@

import java.io.Serializable;
import java.util.LinkedHashMap;
import java.util.Map;

/**
* A metastore client related to a table. All methods of this interface operate on the same specific
Expand All @@ -37,6 +38,12 @@ public interface MetastoreClient extends AutoCloseable {

void markDone(LinkedHashMap<String, String> partitionSpec) throws Exception;

void alterPartition(
LinkedHashMap<String, String> partitionSpec,
Map<String, String> parameters,
long modifyTime)
throws Exception;

/** Factory to create {@link MetastoreClient}. */
interface Factory extends Serializable {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -355,6 +355,14 @@ public class FlinkConnectorOptions {
"You can specify time interval for partition, for example, "
+ "daily partition is '1 d', hourly partition is '1 h'.");

public static final ConfigOption<Duration> PARTITION_IDLE_TIME_TO_REPORT_STATISTIC =
key("partition.idle-time-to-report-statistic")
.durationType()
.defaultValue(Duration.ofHours(1))
.withDescription(
"Set a time duration when a partition has no new data after this time duration, "
+ "start to report the partition statistics to hms.");

public static final ConfigOption<String> CLUSTERING_COLUMNS =
key("sink.clustering.by-columns")
.stringType()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@

import org.apache.paimon.annotation.VisibleForTesting;
import org.apache.paimon.flink.metrics.FlinkMetricRegistry;
import org.apache.paimon.flink.sink.partition.PartitionMarkDone;
import org.apache.paimon.flink.sink.partition.PartitionListeners;
import org.apache.paimon.io.DataFileMeta;
import org.apache.paimon.manifest.ManifestCommittable;
import org.apache.paimon.table.FileStoreTable;
Expand All @@ -43,7 +43,7 @@ public class StoreCommitter implements Committer<Committable, ManifestCommittabl

private final TableCommitImpl commit;
@Nullable private final CommitterMetrics committerMetrics;
@Nullable private final PartitionMarkDone partitionMarkDone;
private final PartitionListeners partitionListeners;

public StoreCommitter(FileStoreTable table, TableCommit commit, Context context) {
this.commit = (TableCommitImpl) commit;
Expand All @@ -56,12 +56,7 @@ public StoreCommitter(FileStoreTable table, TableCommit commit, Context context)
}

try {
this.partitionMarkDone =
PartitionMarkDone.create(
context.streamingCheckpointEnabled(),
context.isRestored(),
context.stateStore(),
table);
this.partitionListeners = PartitionListeners.create(context, table);
} catch (Exception e) {
throw new RuntimeException(e);
}
Expand Down Expand Up @@ -111,29 +106,23 @@ public void commit(List<ManifestCommittable> committables)
throws IOException, InterruptedException {
commit.commitMultiple(committables, false);
calcNumBytesAndRecordsOut(committables);
if (partitionMarkDone != null) {
partitionMarkDone.notifyCommittable(committables);
}
partitionListeners.notifyCommittable(committables);
}

@Override
public int filterAndCommit(
List<ManifestCommittable> globalCommittables, boolean checkAppendFiles) {
int committed = commit.filterAndCommitMultiple(globalCommittables, checkAppendFiles);
if (partitionMarkDone != null) {
partitionMarkDone.notifyCommittable(globalCommittables);
}
partitionListeners.notifyCommittable(globalCommittables);
return committed;
}

@Override
public Map<Long, List<Committable>> groupByCheckpoint(Collection<Committable> committables) {
if (partitionMarkDone != null) {
try {
partitionMarkDone.snapshotState();
} catch (Exception e) {
throw new RuntimeException(e);
}
try {
partitionListeners.snapshotState();
} catch (Exception e) {
throw new RuntimeException(e);
}

Map<Long, List<Committable>> grouped = new HashMap<>();
Expand All @@ -146,9 +135,7 @@ public Map<Long, List<Committable>> groupByCheckpoint(Collection<Committable> co
@Override
public void close() throws Exception {
commit.close();
if (partitionMarkDone != null) {
partitionMarkDone.close();
}
partitionListeners.close();
}

private void calcNumBytesAndRecordsOut(List<ManifestCommittable> committables) {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,115 @@
/*
* 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.paimon.flink.sink.partition;

import org.apache.paimon.Snapshot;
import org.apache.paimon.fs.Path;
import org.apache.paimon.io.DataFileMeta;
import org.apache.paimon.metastore.MetastoreClient;
import org.apache.paimon.table.FileStoreTable;
import org.apache.paimon.table.source.DataSplit;
import org.apache.paimon.table.source.DeletionFile;
import org.apache.paimon.table.source.ScanMode;
import org.apache.paimon.table.source.snapshot.SnapshotReader;
import org.apache.paimon.utils.Preconditions;
import org.apache.paimon.utils.SnapshotManager;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;

import static org.apache.paimon.utils.PartitionPathUtils.extractPartitionSpecFromPath;

/** Action to report the table statistic from the latest snapshot to HMS. */
public class HmsReporter implements Closeable {

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

private final MetastoreClient metastoreClient;
private final SnapshotReader snapshotReader;
private final SnapshotManager snapshotManager;

public HmsReporter(FileStoreTable table, MetastoreClient client) {
this.metastoreClient =
Preconditions.checkNotNull(client, "the metastore client factory is null");
this.snapshotReader = table.newSnapshotReader();
this.snapshotManager = table.snapshotManager();
}

public void report(String partition, long modifyTime) throws Exception {
Snapshot snapshot = snapshotManager.latestSnapshot();
if (snapshot != null) {
LinkedHashMap<String, String> partitionSpec =
extractPartitionSpecFromPath(new Path(partition));
List<DataSplit> splits =
new ArrayList<>(
snapshotReader
.withMode(ScanMode.ALL)
.withPartitionFilter(partitionSpec)
.withSnapshot(snapshot)
.read()
.dataSplits());
long rowCount = 0;
long totalSize = 0;
long fileCount = 0;
for (DataSplit split : splits) {
List<DataFileMeta> fileMetas = split.dataFiles();
rowCount += split.rowCount();
fileCount += fileMetas.size();
for (DataFileMeta fileMeta : fileMetas) {
totalSize += fileMeta.fileSize();
}

if (split.deletionFiles().isPresent()) {
fileCount += split.deletionFiles().get().size();
totalSize +=
split.deletionFiles().get().stream()
.map(DeletionFile::length)
.reduce(0L, Long::sum);
}
}
Map<String, String> statistic = new HashMap<>();
// refer to org.apache.hadoop.hive.common.StatsSetupConst
statistic.put("numFiles", String.valueOf(fileCount));
statistic.put("totalSize", String.valueOf(totalSize));
statistic.put("numRows", String.valueOf(rowCount));
// refer to org.apache.hadoop.hive.metastore.api.hive_metastoreConstants
statistic.put("transient_lastDdlTime", String.valueOf(modifyTime / 1000));

LOG.info("alter partition {} with statistic {}.", partition, statistic);
metastoreClient.alterPartition(partitionSpec, statistic, modifyTime);
}
}

@Override
public void close() throws IOException {
try {
metastoreClient.close();
} catch (Exception e) {
throw new RuntimeException(e);
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,32 @@
/*
* 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.paimon.flink.sink.partition;

import org.apache.paimon.manifest.ManifestCommittable;

import java.io.Closeable;
import java.util.List;

/** The partition listener. */
public interface PartitionListener extends Closeable {

void notifyCommittable(List<ManifestCommittable> committables);

void snapshotState() throws Exception;
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,72 @@
/*
* 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.paimon.flink.sink.partition;

import org.apache.paimon.flink.sink.Committer;
import org.apache.paimon.manifest.ManifestCommittable;
import org.apache.paimon.table.FileStoreTable;
import org.apache.paimon.utils.IOUtils;

import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;

/** Partition listeners. */
public class PartitionListeners implements Closeable {

private final List<PartitionListener> listeners;

private PartitionListeners(List<PartitionListener> listeners) {
this.listeners = listeners;
}

public void notifyCommittable(List<ManifestCommittable> committables) {
for (PartitionListener trigger : listeners) {
trigger.notifyCommittable(committables);
}
}

public void snapshotState() throws Exception {
for (PartitionListener trigger : listeners) {
trigger.snapshotState();
}
}

@Override
public void close() throws IOException {
IOUtils.closeAllQuietly(listeners);
}

public static PartitionListeners create(Committer.Context context, FileStoreTable table)
throws Exception {
List<PartitionListener> listeners = new ArrayList<>();

ReportHmsListener.create(context.isRestored(), context.stateStore(), table)
.ifPresent(listeners::add);
PartitionMarkDone.create(
context.streamingCheckpointEnabled(),
context.isRestored(),
context.stateStore(),
table)
.ifPresent(listeners::add);

return new PartitionListeners(listeners);
}
}
Loading

0 comments on commit 0d064f9

Please sign in to comment.