Skip to content

Commit

Permalink
add MultiTableAppendOnlyCompactionTask
Browse files Browse the repository at this point in the history
  • Loading branch information
wg1026688210 committed May 8, 2024
1 parent 60d2cdf commit 2d1a53d
Show file tree
Hide file tree
Showing 6 changed files with 208 additions and 53 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@

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 @@ -40,22 +39,13 @@ 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, Identifier.EMPTY);
}

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 @@ -82,12 +72,8 @@ public CommitMessage doCompact(AppendOnlyFileStoreWrite write) throws Exception
compactIncrement);
}

public Identifier tableIdentifier() {
return tableIdentifier;
}

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

@Override
Expand All @@ -102,8 +88,7 @@ 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(tableIdentifier, that.tableIdentifier);
&& Objects.equals(compactAfter, that.compactAfter);
}

@Override
Expand All @@ -112,8 +97,7 @@ public String toString() {
"CompactionTask {"
+ "partition = %s, "
+ "compactBefore = %s, "
+ "compactAfter = %s, "
+ "tableIdentifier = %s}",
partition, compactBefore, compactAfter, tableIdentifier);
+ "compactAfter = %s}",
partition, compactBefore, compactAfter);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,61 @@
/*
* 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.append;

import org.apache.paimon.catalog.Identifier;
import org.apache.paimon.data.BinaryRow;
import org.apache.paimon.io.DataFileMeta;

import java.util.List;
import java.util.Objects;

/** Compaction task for multi table . */
public class MultiTableAppendOnlyCompactionTask extends AppendOnlyCompactionTask {
private final Identifier tableIdentifier;

public MultiTableAppendOnlyCompactionTask(
BinaryRow partition, List<DataFileMeta> files, Identifier identifier) {
super(partition, files);
this.tableIdentifier = identifier;
}

public Identifier tableIdentifier() {
return tableIdentifier;
}

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

@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}

MultiTableAppendOnlyCompactionTask that = (MultiTableAppendOnlyCompactionTask) o;
return Objects.equals(partition(), that.partition())
&& Objects.equals(compactBefore(), that.compactBefore())
&& Objects.equals(compactAfter(), that.compactAfter())
&& Objects.equals(tableIdentifier, that.tableIdentifier);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,6 @@
*/
@Public
public class Identifier implements Serializable {
public static final Identifier EMPTY = new Identifier(null, null);

private static final long serialVersionUID = 1L;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@
import org.apache.paimon.io.DataInputView;
import org.apache.paimon.io.DataOutputView;
import org.apache.paimon.io.DataOutputViewStreamWrapper;
import org.apache.paimon.io.IdentifierSerializer;

import java.io.ByteArrayOutputStream;
import java.io.IOException;
Expand All @@ -38,15 +37,12 @@
/** Serializer for {@link AppendOnlyCompactionTask}. */
public class CompactionTaskSerializer implements VersionedSerializer<AppendOnlyCompactionTask> {

private static final int CURRENT_VERSION = 3;
private static final int CURRENT_VERSION = 2;

private final DataFileMetaSerializer dataFileSerializer;

private final IdentifierSerializer identifierSerializer;

public CompactionTaskSerializer() {
this.dataFileSerializer = new DataFileMetaSerializer();
this.identifierSerializer = new IdentifierSerializer();
}

@Override
Expand All @@ -73,7 +69,6 @@ public void serializeList(List<AppendOnlyCompactionTask> list, DataOutputView vi
private void serialize(AppendOnlyCompactionTask task, DataOutputView view) throws IOException {
serializeBinaryRow(task.partition(), view);
dataFileSerializer.serializeList(task.compactBefore(), view);
identifierSerializer.serialize(task.tableIdentifier(), view);
}

@Override
Expand Down Expand Up @@ -108,8 +103,6 @@ private void checkVersion(int version) {

private AppendOnlyCompactionTask deserialize(DataInputView view) throws IOException {
return new AppendOnlyCompactionTask(
deserializeBinaryRow(view),
dataFileSerializer.deserializeList(view),
identifierSerializer.deserialize(view));
deserializeBinaryRow(view), dataFileSerializer.deserializeList(view));
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,117 @@
/*
* 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.table.sink;

import org.apache.paimon.append.MultiTableAppendOnlyCompactionTask;
import org.apache.paimon.data.serializer.VersionedSerializer;
import org.apache.paimon.io.DataFileMetaSerializer;
import org.apache.paimon.io.DataInputDeserializer;
import org.apache.paimon.io.DataInputView;
import org.apache.paimon.io.DataOutputView;
import org.apache.paimon.io.DataOutputViewStreamWrapper;
import org.apache.paimon.io.IdentifierSerializer;

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

import static org.apache.paimon.utils.SerializationUtils.deserializeBinaryRow;
import static org.apache.paimon.utils.SerializationUtils.serializeBinaryRow;

/** Serializer for {@link MultiTableAppendOnlyCompactionTask}. */
public class MultiTableCompactionTaskSerializer
implements VersionedSerializer<MultiTableAppendOnlyCompactionTask> {
private static final int CURRENT_VERSION = 1;

private final DataFileMetaSerializer dataFileSerializer;

private final IdentifierSerializer identifierSerializer;

public MultiTableCompactionTaskSerializer() {
this.dataFileSerializer = new DataFileMetaSerializer();
this.identifierSerializer = new IdentifierSerializer();
}

@Override
public int getVersion() {
return CURRENT_VERSION;
}

@Override
public byte[] serialize(MultiTableAppendOnlyCompactionTask task) throws IOException {
ByteArrayOutputStream out = new ByteArrayOutputStream();
DataOutputViewStreamWrapper view = new DataOutputViewStreamWrapper(out);
serialize(task, view);
return out.toByteArray();
}

private void serialize(MultiTableAppendOnlyCompactionTask task, DataOutputView view)
throws IOException {
serializeBinaryRow(task.partition(), view);
dataFileSerializer.serializeList(task.compactBefore(), view);
identifierSerializer.serialize(task.tableIdentifier(), view);
}

@Override
public MultiTableAppendOnlyCompactionTask deserialize(int version, byte[] serialized)
throws IOException {
checkVersion(version);
DataInputDeserializer view = new DataInputDeserializer(serialized);
return deserialize(view);
}

private MultiTableAppendOnlyCompactionTask deserialize(DataInputView view) throws IOException {
return new MultiTableAppendOnlyCompactionTask(
deserializeBinaryRow(view),
dataFileSerializer.deserializeList(view),
identifierSerializer.deserialize(view));
}

public List<MultiTableAppendOnlyCompactionTask> deserializeList(int version, DataInputView view)
throws IOException {
checkVersion(version);
int length = view.readInt();
List<MultiTableAppendOnlyCompactionTask> list = new ArrayList<>(length);
for (int i = 0; i < length; i++) {
list.add(deserialize(view));
}
return list;
}

public void serializeList(List<MultiTableAppendOnlyCompactionTask> list, DataOutputView view)
throws IOException {
view.writeInt(list.size());
for (MultiTableAppendOnlyCompactionTask commitMessage : list) {
serialize(commitMessage, view);
}
}

private void checkVersion(int version) {
if (version != CURRENT_VERSION) {
throw new UnsupportedOperationException(
"Expecting MultiTableCompactionTaskSerializer version to be "
+ CURRENT_VERSION
+ ", but found "
+ version
+ ".\nCompactionTask is not a compatible data structure. "
+ "Please restart the job afresh (do not recover from savepoint).");
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.paimon.table.sink;

import org.apache.paimon.append.AppendOnlyCompactionTask;
import org.apache.paimon.append.MultiTableAppendOnlyCompactionTask;
import org.apache.paimon.catalog.Identifier;

import org.junit.jupiter.api.Test;
Expand All @@ -29,32 +30,32 @@
import static org.apache.paimon.mergetree.compact.MergeTreeCompactManagerTest.row;
import static org.assertj.core.api.Assertions.assertThat;

/** Tests for {@link CompactionTaskSerializer}. */
/** Tests for {@link CompactionTaskSerializer} and {@link MultiTableCompactionTaskSerializer}. */
public class CompactionTaskSerializerTest {

@Test
public void testCompactionTaskSerializer() throws IOException {
{
CompactionTaskSerializer serializer = new CompactionTaskSerializer();
AppendOnlyCompactionTask task =
new AppendOnlyCompactionTask(row(0), randomNewFilesIncrement().newFiles());

byte[] bytes = serializer.serialize(task);
AppendOnlyCompactionTask task1 = serializer.deserialize(serializer.getVersion(), bytes);
assertThat(task).isEqualTo(task1);
}

{
CompactionTaskSerializer serializer = new CompactionTaskSerializer();
AppendOnlyCompactionTask task =
new AppendOnlyCompactionTask(
row(0),
randomNewFilesIncrement().newFiles(),
Identifier.create("db", "table"));

byte[] bytes = serializer.serialize(task);
AppendOnlyCompactionTask task1 = serializer.deserialize(serializer.getVersion(), bytes);
assertThat(task).isEqualTo(task1);
}
CompactionTaskSerializer serializer = new CompactionTaskSerializer();
AppendOnlyCompactionTask task =
new AppendOnlyCompactionTask(row(0), randomNewFilesIncrement().newFiles());

byte[] bytes = serializer.serialize(task);
AppendOnlyCompactionTask task1 = serializer.deserialize(serializer.getVersion(), bytes);
assertThat(task).isEqualTo(task1);
}

@Test
public void testMultiTableCompactionTaskSerializer() throws IOException {
MultiTableCompactionTaskSerializer serializer = new MultiTableCompactionTaskSerializer();
MultiTableAppendOnlyCompactionTask task =
new MultiTableAppendOnlyCompactionTask(
row(0),
randomNewFilesIncrement().newFiles(),
Identifier.create("db", "table"));

byte[] bytes = serializer.serialize(task);
MultiTableAppendOnlyCompactionTask task1 =
serializer.deserialize(serializer.getVersion(), bytes);
assertThat(task).isEqualTo(task1);
}
}

0 comments on commit 2d1a53d

Please sign in to comment.