Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,8 @@

package org.apache.paimon.flink.sink;

import org.apache.flink.metrics.groups.OperatorIOMetricGroup;

import java.io.IOException;
import java.util.Collection;
import java.util.List;
Expand All @@ -37,7 +39,8 @@ GlobalCommitT combine(long checkpointId, long watermark, List<CommitT> committab
throws IOException;

/** Commits the given {@link GlobalCommitT}. */
void commit(List<GlobalCommitT> globalCommittables) throws IOException, InterruptedException;
void commit(List<GlobalCommitT> globalCommittables, OperatorIOMetricGroup metricGroup)
throws IOException, InterruptedException;

/**
* Filter out all {@link GlobalCommitT} which have committed, and commit the remaining {@link
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -154,7 +154,7 @@ public void notifyCheckpointComplete(long checkpointId) throws Exception {
private void commitUpToCheckpoint(long checkpointId) throws Exception {
NavigableMap<Long, GlobalCommitT> headMap =
committablesPerCheckpoint.headMap(checkpointId, true);
committer.commit(committables(headMap));
committer.commit(committables(headMap), getMetricGroup().getIOMetricGroup());
headMap.clear();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,11 +18,17 @@

package org.apache.paimon.flink.sink;

import org.apache.paimon.annotation.VisibleForTesting;
import org.apache.paimon.io.DataFileMeta;
import org.apache.paimon.manifest.ManifestCommittable;
import org.apache.paimon.table.sink.CommitMessage;
import org.apache.paimon.table.sink.CommitMessageImpl;
import org.apache.paimon.table.sink.TableCommit;
import org.apache.paimon.table.sink.TableCommitImpl;

import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.metrics.groups.OperatorIOMetricGroup;

import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
Expand Down Expand Up @@ -60,9 +66,12 @@ public ManifestCommittable combine(
}

@Override
public void commit(List<ManifestCommittable> committables)
public void commit(List<ManifestCommittable> committables, OperatorIOMetricGroup metricGroup)
throws IOException, InterruptedException {
commit.commitMultiple(committables);
Tuple2<Long, Long> numBytesAndRecords = calcDataBytesAndRecordsSend(committables);
metricGroup.getNumBytesOutCounter().inc(numBytesAndRecords.f0);
metricGroup.getNumRecordsOutCounter().inc(numBytesAndRecords.f1);
}

@Override
Expand All @@ -83,4 +92,32 @@ public Map<Long, List<Committable>> groupByCheckpoint(Collection<Committable> co
public void close() throws Exception {
commit.close();
}

@VisibleForTesting
static Tuple2<Long, Long> calcDataBytesAndRecordsSend(List<ManifestCommittable> committables) {
long bytesSend = 0;
long recordsSend = 0;
for (ManifestCommittable committable : committables) {
List<CommitMessage> commitMessages = committable.fileCommittables();
for (CommitMessage commitMessage : commitMessages) {
long dataFileSizeInc =
calcTotalFileSize(
((CommitMessageImpl) commitMessage).newFilesIncrement().newFiles());
long dataFileRowCountInc =
calcTotalFileRowCount(
((CommitMessageImpl) commitMessage).newFilesIncrement().newFiles());
bytesSend += dataFileSizeInc;
recordsSend += dataFileRowCountInc;
}
}
return Tuple2.of(bytesSend, recordsSend);
}

private static long calcTotalFileSize(List<DataFileMeta> files) {
return files.stream().mapToLong(f -> f.fileSize()).reduce(Long::sum).orElse(0);
}

private static long calcTotalFileRowCount(List<DataFileMeta> files) {
return files.stream().mapToLong(f -> f.rowCount()).reduce(Long::sum).orElse(0);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@
import org.apache.paimon.table.sink.CommitMessage;

import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.metrics.groups.OperatorIOMetricGroup;

import java.io.IOException;
import java.util.ArrayList;
Expand Down Expand Up @@ -84,7 +85,8 @@ public WrappedManifestCommittable combine(
}

@Override
public void commit(List<WrappedManifestCommittable> committables)
public void commit(
List<WrappedManifestCommittable> committables, OperatorIOMetricGroup metricGroup)
throws IOException, InterruptedException {

// key by table id
Expand All @@ -94,7 +96,7 @@ public void commit(List<WrappedManifestCommittable> committables)
Identifier tableId = entry.getKey();
List<ManifestCommittable> committableList = entry.getValue();
StoreCommitter committer = getStoreCommitter(tableId);
committer.commit(committableList);
committer.commit(committableList, metricGroup);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import org.apache.paimon.manifest.ManifestCommittableSerializer;
import org.apache.paimon.table.FileStoreTable;
import org.apache.paimon.table.sink.CommitMessage;
import org.apache.paimon.table.sink.StreamTableCommit;
import org.apache.paimon.table.sink.StreamTableWrite;
import org.apache.paimon.table.sink.StreamWriteBuilder;
import org.apache.paimon.utils.SnapshotManager;
Expand All @@ -35,6 +36,7 @@
import org.apache.flink.api.common.state.ListState;
import org.apache.flink.api.common.state.ListStateDescriptor;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
import org.apache.flink.runtime.state.StateInitializationContext;
import org.apache.flink.streaming.api.watermark.Watermark;
Expand All @@ -45,6 +47,7 @@
import org.junit.jupiter.api.Test;

import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.UUID;

Expand Down Expand Up @@ -322,6 +325,31 @@ public void testWatermarkCommit() throws Exception {
assertThat(table.snapshotManager().latestSnapshot().watermark()).isEqualTo(1024L);
}

@Test
public void testCalcDataBytesSend() throws Exception {
FileStoreTable table = createFileStoreTable();

StreamTableWrite write = table.newWrite(initialCommitUser);
StreamTableCommit commit = table.newCommit(initialCommitUser);

write.write(GenericRow.of(1, 10L));
write.write(GenericRow.of(1, 20L));
List<CommitMessage> committable = write.prepareCommit(false, 0);

commit.commit(0, committable);

ManifestCommittable manifestCommittable = new ManifestCommittable(0);
for (CommitMessage commitMessage : committable) {
manifestCommittable.addFileCommittable(commitMessage);
}
write.close();
Tuple2<Long, Long> numBytesAndRecords =
StoreCommitter.calcDataBytesAndRecordsSend(
new ArrayList<>(Arrays.asList(manifestCommittable)));
assertThat(numBytesAndRecords.f0).isEqualTo(275);
assertThat(numBytesAndRecords.f1).isEqualTo(2);
}

// ------------------------------------------------------------------------
// Test utils
// ------------------------------------------------------------------------
Expand Down