Skip to content

Commit

Permalink
compaction
Browse files Browse the repository at this point in the history
  • Loading branch information
wg1026688210 committed Feb 5, 2024
1 parent dc1da67 commit eab9112
Show file tree
Hide file tree
Showing 17 changed files with 949 additions and 250 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

package org.apache.paimon.append;

import org.apache.paimon.catalog.Identifier;
import org.apache.paimon.data.BinaryRow;
import org.apache.paimon.io.CompactIncrement;
import org.apache.paimon.io.DataFileMeta;
Expand All @@ -39,13 +40,22 @@ public class AppendOnlyCompactionTask {
private final List<DataFileMeta> compactBefore;
private final List<DataFileMeta> compactAfter;

private final Identifier tableIdentifier;

public AppendOnlyCompactionTask(BinaryRow partition, List<DataFileMeta> files) {
this(partition, files, null);
}

public AppendOnlyCompactionTask(
BinaryRow partition, List<DataFileMeta> files, Identifier identifier) {

Preconditions.checkArgument(
files != null && files.size() > 1,
"AppendOnlyCompactionTask need more than one file input.");
this.partition = partition;
compactBefore = new ArrayList<>(files);
compactAfter = new ArrayList<>();
this.tableIdentifier = identifier;
}

public BinaryRow partition() {
Expand All @@ -72,8 +82,12 @@ public CommitMessage doCompact(AppendOnlyFileStoreWrite write) throws Exception
compactIncrement);
}

public Identifier tableIdentifier() {
return tableIdentifier;
}

public int hashCode() {
return Objects.hash(partition, compactBefore, compactAfter);
return Objects.hash(partition, compactBefore, compactAfter, tableIdentifier);
}

@Override
Expand All @@ -88,7 +102,8 @@ public boolean equals(Object o) {
AppendOnlyCompactionTask that = (AppendOnlyCompactionTask) o;
return Objects.equals(partition, that.partition)
&& Objects.equals(compactBefore, that.compactBefore)
&& Objects.equals(compactAfter, that.compactAfter);
&& Objects.equals(compactAfter, that.compactAfter)
&& Objects.equals(tableIdentifier, that.tableIdentifier);
}

@Override
Expand All @@ -97,7 +112,8 @@ public String toString() {
"CompactionTask {"
+ "partition = %s, "
+ "compactBefore = %s, "
+ "compactAfter = %s}",
partition, compactBefore, compactAfter);
+ "compactAfter = %s, "
+ "tableIdentifier = %s}",
partition, compactBefore, compactAfter, tableIdentifier);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.paimon.flink.action;

import org.apache.paimon.CoreOptions;
import org.apache.paimon.append.AppendOnlyCompactionTask;
import org.apache.paimon.catalog.Catalog;
import org.apache.paimon.catalog.Identifier;
import org.apache.paimon.flink.FlinkConnectorOptions;
Expand Down Expand Up @@ -197,15 +198,26 @@ private void buildForCombinedMode() {
includingPattern,
excludingPattern,
tableOptions.get(CoreOptions.CONTINUOUS_DISCOVERY_INTERVAL).toMillis());
DataStream<RowData> source =
sourceBuilder.withEnv(env).withContinuousMode(isStreaming).build();
DataStream<RowData> fixedAndDynamicBucketsTable =
sourceBuilder
.withEnv(env)
.withContinuousMode(isStreaming)
.buildForFixedAndDynamicBucketsTable();

DataStream<RowData> partitioned =
partition(
source,
fixedAndDynamicBucketsTable,
new BucketsRowChannelComputer(),
tableOptions.get(FlinkConnectorOptions.SINK_PARALLELISM));
new MultiTablesCompactorSink(catalogLoader(), tableOptions).sinkFrom(partitioned);

// unaware table
DataStream<AppendOnlyCompactionTask> unawareBucketsTable =
sourceBuilder
.withEnv(env)
.withContinuousMode(isStreaming)
.buildForUnawareBucketsTable();

new MultiTablesCompactorSink(catalogLoader(), tableOptions).sinkFrom(partitioned,unawareBucketsTable);
}

private void buildForTraditionalCompaction(
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,124 @@
/*
* 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.compact;

import org.apache.paimon.annotation.VisibleForTesting;
import org.apache.paimon.append.AppendOnlyCompactionTask;
import org.apache.paimon.flink.sink.Committable;
import org.apache.paimon.operation.AppendOnlyFileStoreWrite;
import org.apache.paimon.table.FileStoreTable;
import org.apache.paimon.table.sink.CommitMessage;
import org.apache.paimon.table.sink.TableCommitImpl;

import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;

import java.io.IOException;
import java.util.ArrayList;
import java.util.LinkedList;
import java.util.List;
import java.util.Queue;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import java.util.function.Supplier;
import java.util.stream.Collectors;

/**
* The help Class for the Compaction of unware bucket append table to execute {@link
* AppendOnlyCompactionTask}.
*/
public class UnwareBucketCompactionHelper {
private final FileStoreTable table;
private final String commitUser;

private final transient AppendOnlyFileStoreWrite write;

protected final transient Queue<Future<CommitMessage>> result;

private final transient Supplier<ExecutorService> compactExecutorsupplier;

public UnwareBucketCompactionHelper(
FileStoreTable table,
String commitUser,
Supplier<ExecutorService> lazyCompactExecutor) {
this.table = table;
this.commitUser = commitUser;
this.write = (AppendOnlyFileStoreWrite) table.store().newWrite(commitUser);
this.result = new LinkedList<>();
this.compactExecutorsupplier = lazyCompactExecutor;
}

public void processElement(StreamRecord<AppendOnlyCompactionTask> element) throws Exception {
AppendOnlyCompactionTask task = element.getValue();
result.add(compactExecutorsupplier.get().submit(() -> task.doCompact(write)));
}

public void close() throws Exception {
shutdown();
}

@VisibleForTesting
void shutdown() throws Exception {

List<CommitMessage> messages = new ArrayList<>();
for (Future<CommitMessage> resultFuture : result) {
if (!resultFuture.isDone()) {
// the later tasks should be stopped running
break;
}
try {
messages.add(resultFuture.get());
} catch (Exception exception) {
// exception should already be handled
}
}
if (messages.isEmpty()) {
return;
}

try (TableCommitImpl tableCommit = table.newCommit(commitUser)) {
tableCommit.abort(messages);
}
}

public List<Committable> prepareCommit(boolean waitCompaction, long checkpointId)
throws IOException {
List<CommitMessage> tempList = new ArrayList<>();
try {
while (!result.isEmpty()) {
Future<CommitMessage> future = result.peek();
if (!future.isDone() && !waitCompaction) {
break;
}
result.poll();
tempList.add(future.get());
}
return tempList.stream()
.map(s -> new Committable(checkpointId, Committable.Kind.FILE, s))
.collect(Collectors.toList());
} catch (InterruptedException e) {
throw new RuntimeException("Interrupted while waiting tasks done.", e);
} catch (Exception e) {
throw new RuntimeException("Encountered an error while do compaction", e);
}
}

public Iterable<Future<CommitMessage>> result() {
return result;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

package org.apache.paimon.flink.sink;

import org.apache.paimon.append.AppendOnlyCompactionTask;
import org.apache.paimon.catalog.Catalog;
import org.apache.paimon.flink.VersionedSerializerWrapper;
import org.apache.paimon.flink.utils.StreamExecutionEnvironmentUtils;
Expand Down Expand Up @@ -58,55 +59,80 @@ public class MultiTablesCompactorSink implements Serializable {

private final Options options;

public MultiTablesCompactorSink(Catalog.Loader catalogLoader, Options options) {
public MultiTablesCompactorSink(
Catalog.Loader catalogLoader,
Options options) {
this.catalogLoader = catalogLoader;
this.ignorePreviousFiles = false;
this.options = options;
}

public DataStreamSink<?> sinkFrom(DataStream<RowData> input) {
public DataStreamSink<?> sinkFrom(
DataStream<RowData> input, DataStream<AppendOnlyCompactionTask> unawareTableSource) {
// This commitUser is valid only for new jobs.
// After the job starts, this commitUser will be recorded into the states of write and
// commit operators.
// When the job restarts, commitUser will be recovered from states and this value is
// ignored.
String initialCommitUser = UUID.randomUUID().toString();
return sinkFrom(input, initialCommitUser);
return sinkFrom(input, unawareTableSource, initialCommitUser);
}

public DataStreamSink<?> sinkFrom(DataStream<RowData> input, String initialCommitUser) {
public DataStreamSink<?> sinkFrom(
DataStream<RowData> fixAndDynamicTableSource,
DataStream<AppendOnlyCompactionTask> unawareTableSource,
String initialCommitUser) {
// do the actually writing action, no snapshot generated in this stage
SingleOutputStreamOperator<MultiTableCommittable> written =
doWrite(input, initialCommitUser, input.getParallelism());
DataStream<MultiTableCommittable> written =
doWrite(fixAndDynamicTableSource, unawareTableSource, initialCommitUser);

// commit the committable to generate a new snapshot
return doCommit(written, initialCommitUser);
}

public SingleOutputStreamOperator<MultiTableCommittable> doWrite(
DataStream<RowData> input, String commitUser, Integer parallelism) {
StreamExecutionEnvironment env = input.getExecutionEnvironment();
public DataStream<MultiTableCommittable> doWrite(
DataStream<RowData> fixAndDynamicTableSource,
DataStream<AppendOnlyCompactionTask> unawareTableSource,
String commitUser) {
StreamExecutionEnvironment env = fixAndDynamicTableSource.getExecutionEnvironment();
boolean isStreaming =
StreamExecutionEnvironmentUtils.getConfiguration(env)
.get(ExecutionOptions.RUNTIME_MODE)
== RuntimeExecutionMode.STREAMING;

SingleOutputStreamOperator<MultiTableCommittable> written =
input.transform(
WRITER_NAME,
SingleOutputStreamOperator<MultiTableCommittable> fixAndDynamicTableRewriter =
fixAndDynamicTableSource
.transform(
String.format(
"%s-%s",
"The table of fix and dynamic bucket", WRITER_NAME),
new MultiTableCommittableTypeInfo(),
createWriteOperator(
env.getCheckpointConfig(), isStreaming, commitUser))
.setParallelism(parallelism == null ? input.getParallelism() : parallelism);
.setParallelism(fixAndDynamicTableSource.getParallelism());

SingleOutputStreamOperator<MultiTableCommittable> unawareTableRewriter =
unawareTableSource
.transform(
String.format("%s-%s", "The table of unaware bucket", WRITER_NAME),
new MultiTableCommittableTypeInfo(),
new UnwaredMultiTableCompactionWorkerOperator(
catalogLoader, commitUser, options))
.setParallelism(unawareTableSource.getParallelism());

if (!isStreaming) {
assertBatchConfiguration(env, written.getParallelism());
assertBatchConfiguration(env, fixAndDynamicTableRewriter.getParallelism());
assertBatchConfiguration(env, unawareTableRewriter.getParallelism());
}

if (options.get(SINK_USE_MANAGED_MEMORY)) {
declareManagedMemory(written, options.get(SINK_MANAGED_WRITER_BUFFER_MEMORY));
declareManagedMemory(
fixAndDynamicTableRewriter, options.get(SINK_MANAGED_WRITER_BUFFER_MEMORY));
declareManagedMemory(
unawareTableRewriter, options.get(SINK_MANAGED_WRITER_BUFFER_MEMORY));
}
return written;

return fixAndDynamicTableRewriter.union(unawareTableRewriter);
}

protected DataStreamSink<?> doCommit(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,7 @@ public static DataStreamSink<?> sink(
@Override
protected OneInputStreamOperator<AppendOnlyCompactionTask, Committable> createWriteOperator(
StoreSinkWrite.Provider writeProvider, String commitUser) {
return new AppendOnlyTableCompactionWorkerOperator(table, commitUser);
return new UnwaredSingleCompactionWorkerOperator(table, commitUser);
}

@Override
Expand Down
Loading

0 comments on commit eab9112

Please sign in to comment.