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
25 changes: 25 additions & 0 deletions docs/content/flink/procedures.md
Original file line number Diff line number Diff line change
Expand Up @@ -344,6 +344,31 @@ All available procedures are listed below.
CALL remove_orphan_files(`table` => 'default.T', older_than => '2023-10-31 12:00:00', dry_run => false, parallelism => '5', mode => 'local')
</td>
</tr>
<tr>
<td>remove_unexisting_files</td>
<td>
-- Use named argument<br/>
CALL [catalog.]sys.remove_unexisting_files(`table` => 'identifier', dry_run => 'dryRun', parallelism => 'parallelism') <br/><br/>
-- Use indexed argument<br/>
CALL [catalog.]sys.remove_unexisting_files('identifier')<br/><br/>
CALL [catalog.]sys.remove_unexisting_files('identifier', 'dryRun', 'parallelism')
</td>
<td>
Procedure to remove unexisting data files from manifest entries. See <a href="https://paimon.apache.org/docs/master/api/java/org/apache/paimon/flink/action/RemoveUnexistingFilesAction.html">Java docs</a> for detailed use cases. Arguments:
<li>identifier: the target table identifier. Cannot be empty, you can use database_name.* to clean whole database.</li>
<li>dryRun (optional): only check what files will be removed, but not really remove them. Default is false.</li>
<li>parallelism (optional): number of parallelisms to check files in the manifests.</li>
<br>
Note that user is on his own risk using this procedure, which may cause data loss when used outside from the use cases listed in Java docs.
</td>
<td>
-- remove unexisting data files in the table `mydb.myt`
CALL sys.remove_unexisting_files(`table` => 'mydb.myt')
<br>
-- only check what files will be removed, but not really remove them (dry run)
CALL sys.remove_unexisting_files(`table` => 'mydb.myt', `dry_run` = true)
</td>
</tr>
<tr>
<td>reset_consumer</td>
<td>
Expand Down
18 changes: 18 additions & 0 deletions docs/content/spark/procedures.md
Original file line number Diff line number Diff line change
Expand Up @@ -256,6 +256,24 @@ This section introduce all available spark procedures about paimon.
CALL sys.remove_orphan_files(table => 'default.T', older_than => '2023-10-31 12:00:00', dry_run => true, parallelism => '5', mode => 'local')
</td>
</tr>
<tr>
<td>remove_unexisting_files</td>
<td>
Procedure to remove unexisting data files from manifest entries. See <a href="https://paimon.apache.org/docs/master/api/java/org/apache/paimon/flink/action/RemoveUnexistingFilesAction.html">Java docs</a> for detailed use cases. Arguments:
<li>identifier: the target table identifier. Cannot be empty, you can use database_name.* to clean whole database.</li>
<li>dryRun (optional): only check what files will be removed, but not really remove them. Default is false.</li>
<li>parallelism (optional): number of parallelisms to check files in the manifests.</li>
<br>
Note that user is on his own risk using this procedure, which may cause data loss when used outside from the use cases listed in Java docs.
</td>
<td>
-- remove unexisting data files in the table `mydb.myt`
CALL sys.remove_unexisting_files(table => 'mydb.myt')
<br>
-- only check what files will be removed, but not really remove them (dry run)
CALL sys.remove_unexisting_files(table => 'mydb.myt', dry_run = true)
</td>
</tr>
<tr>
<td>repair</td>
<td>
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,104 @@
/*
* 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.operation;

import org.apache.paimon.data.BinaryRow;
import org.apache.paimon.fs.Path;
import org.apache.paimon.io.DataFileMeta;
import org.apache.paimon.io.DataFilePathFactory;
import org.apache.paimon.table.FileStoreTable;
import org.apache.paimon.table.source.DataSplit;
import org.apache.paimon.table.source.Split;
import org.apache.paimon.utils.FileStorePathFactory;
import org.apache.paimon.utils.ThreadPoolUtils;

import java.io.IOException;
import java.io.UncheckedIOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ThreadPoolExecutor;

import static org.apache.paimon.utils.ThreadPoolUtils.createCachedThreadPool;

/** List what data files recorded in manifests are missing from the filesystem. */
public class ListUnexistingFiles {

private final FileStoreTable table;
private final FileStorePathFactory pathFactory;
private final ThreadPoolExecutor executor;

public ListUnexistingFiles(FileStoreTable table) {
this.table = table;
this.pathFactory = table.store().pathFactory();
this.executor =
createCachedThreadPool(
table.coreOptions().deleteFileThreadNum(), "LIST_UNEXISTING_FILES");
}

public Map<Integer, Map<String, DataFileMeta>> list(BinaryRow partition) throws Exception {
Map<Integer, Map<String, DataFileMeta>> result = new HashMap<>();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe it is better use UnexistingFiles replace result.

List<Split> splits =
table.newScan()
.withPartitionFilter(Collections.singletonList(partition))
.plan()
.splits();
Iterator<ListResult> it =
ThreadPoolUtils.randomlyExecuteSequentialReturn(
executor, split -> listFilesInDataSplit((DataSplit) split), splits);
while (it.hasNext()) {
ListResult item = it.next();
result.computeIfAbsent(item.bucket, k -> new HashMap<>()).put(item.path, item.meta);
}
return result;
}

private List<ListResult> listFilesInDataSplit(DataSplit dataSplit) {
List<ListResult> results = new ArrayList<>();
DataFilePathFactory dataFilePathFactory =
pathFactory.createDataFilePathFactory(dataSplit.partition(), dataSplit.bucket());
for (DataFileMeta meta : dataSplit.dataFiles()) {
Path path = dataFilePathFactory.toPath(meta);
try {
if (!table.fileIO().exists(path)) {
results.add(new ListResult(dataSplit.bucket(), path.toString(), meta));
}
} catch (IOException e) {
throw new UncheckedIOException("Cannot determine if file " + path + " exists.", e);
}
}
return results;
}

private static class ListResult {

private final int bucket;
private final String path;
private final DataFileMeta meta;

private ListResult(int bucket, String path, DataFileMeta meta) {
this.bucket = bucket;
this.path = path;
this.meta = meta;
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,181 @@
/*
* 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.operation;

import org.apache.paimon.CoreOptions;
import org.apache.paimon.catalog.FileSystemCatalog;
import org.apache.paimon.catalog.Identifier;
import org.apache.paimon.data.BinaryRow;
import org.apache.paimon.data.BinaryRowWriter;
import org.apache.paimon.data.GenericRow;
import org.apache.paimon.fs.FileStatus;
import org.apache.paimon.fs.Path;
import org.apache.paimon.fs.local.LocalFileIO;
import org.apache.paimon.io.DataFileMeta;
import org.apache.paimon.schema.Schema;
import org.apache.paimon.table.FileStoreTable;
import org.apache.paimon.table.sink.TableCommitImpl;
import org.apache.paimon.table.sink.TableWriteImpl;
import org.apache.paimon.types.DataType;
import org.apache.paimon.types.DataTypes;
import org.apache.paimon.types.RowType;

import org.junit.jupiter.api.io.TempDir;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.ValueSource;

import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.UUID;
import java.util.concurrent.ThreadLocalRandom;
import java.util.function.Function;
import java.util.stream.Collectors;

import static org.assertj.core.api.Assertions.assertThat;

/** Tests for {@link ListUnexistingFiles}. */
public class ListUnexistingFilesTest {

@TempDir java.nio.file.Path tempDir;

@ParameterizedTest
@ValueSource(ints = {-1, 3})
public void testListFiles(int bucket) throws Exception {
int numPartitions = 2;
int numFiles = 10;
int[] numDeletes = new int[numPartitions];
FileStoreTable table =
prepareRandomlyDeletedTable(
tempDir.toString(), "mydb", "t", bucket, numFiles, numDeletes);

Function<Integer, BinaryRow> binaryRow =
i -> {
BinaryRow b = new BinaryRow(1);
BinaryRowWriter writer = new BinaryRowWriter(b);
writer.writeInt(0, i);
writer.complete();
return b;
};

ListUnexistingFiles operation = new ListUnexistingFiles(table);
for (int i = 0; i < numPartitions; i++) {
Map<Integer, Map<String, DataFileMeta>> result = operation.list(binaryRow.apply(i));
assertThat(result.values().stream().mapToInt(Map::size).sum()).isEqualTo(numDeletes[i]);
}
}

public static FileStoreTable prepareRandomlyDeletedTable(
String warehouse,
String databaseName,
String tableName,
int bucket,
int numFiles,
int[] numDeletes)
throws Exception {
RowType rowType =
RowType.of(
new DataType[] {DataTypes.INT(), DataTypes.INT(), DataTypes.BIGINT()},
new String[] {"pt", "id", "v"});
Map<String, String> options = new HashMap<>();
options.put(CoreOptions.BUCKET.key(), String.valueOf(bucket));
options.put(CoreOptions.WRITE_ONLY.key(), "true");
if (bucket > 0) {
options.put(CoreOptions.BUCKET_KEY.key(), "id");
}
FileStoreTable table =
createPaimonTable(
warehouse,
databaseName,
tableName,
rowType,
Collections.singletonList("pt"),
options);

String commitUser = UUID.randomUUID().toString();
TableWriteImpl<?> write = table.newWrite(commitUser);
TableCommitImpl commit = table.newCommit(commitUser);

ThreadLocalRandom random = ThreadLocalRandom.current();
int numPartitions = numDeletes.length;
for (int i = 0; i < numPartitions; i++) {
numDeletes[i] = random.nextInt(0, numFiles + 1);
}

int identifier = 0;
for (int i = 0; i < numPartitions; i++) {
for (int j = 0; j < numFiles; j++) {
write.write(GenericRow.of(i, random.nextInt(), random.nextLong()));
identifier++;
commit.commit(identifier, write.prepareCommit(false, identifier));
}
}

write.close();
commit.close();

for (int i = 0; i < numPartitions; i++) {
LocalFileIO fileIO = LocalFileIO.create();
List<Path> paths = new ArrayList<>();
for (int j = 0; j < Math.max(1, bucket); j++) {
Path path = new Path(table.location(), "pt=" + i + "/bucket-" + j);
paths.addAll(
Arrays.stream(fileIO.listStatus(path))
.map(FileStatus::getPath)
.collect(Collectors.toList()));
}
Collections.shuffle(paths);
for (int j = 0; j < numDeletes[i]; j++) {
fileIO.deleteQuietly(paths.get(j));
}
}

return table;
}

private static FileStoreTable createPaimonTable(
String warehouse,
String databaseName,
String tableName,
RowType rowType,
List<String> partitionKeys,
Map<String, String> customOptions)
throws Exception {
LocalFileIO fileIO = LocalFileIO.create();
Path path = new Path(warehouse);

Schema schema =
new Schema(
rowType.getFields(),
partitionKeys,
Collections.emptyList(),
customOptions,
"");

try (FileSystemCatalog paimonCatalog = new FileSystemCatalog(fileIO, path)) {
paimonCatalog.createDatabase(databaseName, true);
Identifier paimonIdentifier = Identifier.create(databaseName, tableName);
paimonCatalog.createTable(paimonIdentifier, schema, false);
return (FileStoreTable) paimonCatalog.getTable(paimonIdentifier);
}
}
}
Loading
Loading