Skip to content

Commit

Permalink
[core] System files table distributed execute remote (#4204)
Browse files Browse the repository at this point in the history
  • Loading branch information
leaves12138 authored Sep 23, 2024
1 parent 8652531 commit 350990a
Show file tree
Hide file tree
Showing 2 changed files with 43 additions and 31 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

package org.apache.paimon.table.system;

import org.apache.paimon.data.BinaryRow;
import org.apache.paimon.data.BinaryString;
import org.apache.paimon.data.GenericRow;
import org.apache.paimon.data.InternalArray;
Expand Down Expand Up @@ -47,6 +48,7 @@
import org.apache.paimon.table.source.Split;
import org.apache.paimon.table.source.TableRead;
import org.apache.paimon.table.source.TableScan;
import org.apache.paimon.table.source.snapshot.SnapshotReader;
import org.apache.paimon.types.BigIntType;
import org.apache.paimon.types.DataField;
import org.apache.paimon.types.DataType;
Expand Down Expand Up @@ -75,6 +77,7 @@
import java.util.TreeMap;
import java.util.function.Function;
import java.util.function.Supplier;
import java.util.stream.Collectors;

import static org.apache.paimon.catalog.Catalog.SYSTEM_TABLE_SPLITTER;

Expand Down Expand Up @@ -134,7 +137,7 @@ public List<String> primaryKeys() {

@Override
public InnerTableScan newScan() {
return new FilesScan();
return new FilesScan(storeTable);
}

@Override
Expand All @@ -154,6 +157,12 @@ private static class FilesScan extends ReadOnceTableScan {
@Nullable private LeafPredicate bucketPredicate;
@Nullable private LeafPredicate levelPredicate;

private final FileStoreTable fileStoreTable;

public FilesScan(FileStoreTable fileStoreTable) {
this.fileStoreTable = fileStoreTable;
}

@Override
public InnerTableScan withFilter(Predicate pushdown) {
if (pushdown == null) {
Expand All @@ -170,23 +179,46 @@ public InnerTableScan withFilter(Predicate pushdown) {

@Override
public Plan innerPlan() {
SnapshotReader snapshotReader = fileStoreTable.newSnapshotReader();
if (partitionPredicate != null && partitionPredicate.function() instanceof Equal) {
String partitionStr = partitionPredicate.literals().get(0).toString();
if (partitionStr.startsWith("[")) {
partitionStr = partitionStr.substring(1);
}
if (partitionStr.endsWith("]")) {
partitionStr = partitionStr.substring(0, partitionStr.length() - 1);
}
String[] partFields = partitionStr.split(", ");
LinkedHashMap<String, String> partSpec = new LinkedHashMap<>();
List<String> partitionKeys = fileStoreTable.partitionKeys();
if (partitionKeys.size() != partFields.length) {
return Collections::emptyList;
}
for (int i = 0; i < partitionKeys.size(); i++) {
partSpec.put(partitionKeys.get(i), partFields[i]);
}
snapshotReader.withPartitionFilter(partSpec);
// TODO support range?
}

return () ->
Collections.singletonList(
new FilesSplit(partitionPredicate, bucketPredicate, levelPredicate));
snapshotReader.partitions().stream()
.map(p -> new FilesSplit(p, bucketPredicate, levelPredicate))
.collect(Collectors.toList());
}
}

private static class FilesSplit extends SingletonSplit {

@Nullable private final LeafPredicate partitionPredicate;
@Nullable private final BinaryRow partition;
@Nullable private final LeafPredicate bucketPredicate;
@Nullable private final LeafPredicate levelPredicate;

private FilesSplit(
@Nullable LeafPredicate partitionPredicate,
@Nullable BinaryRow partition,
@Nullable LeafPredicate bucketPredicate,
@Nullable LeafPredicate levelPredicate) {
this.partitionPredicate = partitionPredicate;
this.partition = partition;
this.bucketPredicate = bucketPredicate;
this.levelPredicate = levelPredicate;
}
Expand All @@ -200,14 +232,14 @@ public boolean equals(Object o) {
return false;
}
FilesSplit that = (FilesSplit) o;
return Objects.equals(partitionPredicate, that.partitionPredicate)
return Objects.equals(partition, that.partition)
&& Objects.equals(bucketPredicate, that.bucketPredicate)
&& Objects.equals(this.levelPredicate, that.levelPredicate);
}

@Override
public int hashCode() {
return Objects.hash(partitionPredicate, bucketPredicate, levelPredicate);
return Objects.hash(partition, bucketPredicate, levelPredicate);
}

public List<Split> splits(FileStoreTable storeTable) {
Expand All @@ -216,27 +248,8 @@ public List<Split> splits(FileStoreTable storeTable) {

private TableScan.Plan tablePlan(FileStoreTable storeTable) {
InnerTableScan scan = storeTable.newScan();
if (partitionPredicate != null) {
if (partitionPredicate.function() instanceof Equal) {
String partitionStr = partitionPredicate.literals().get(0).toString();
if (partitionStr.startsWith("[")) {
partitionStr = partitionStr.substring(1);
}
if (partitionStr.endsWith("]")) {
partitionStr = partitionStr.substring(0, partitionStr.length() - 1);
}
String[] partFields = partitionStr.split(", ");
LinkedHashMap<String, String> partSpec = new LinkedHashMap<>();
List<String> partitionKeys = storeTable.partitionKeys();
if (partitionKeys.size() != partFields.length) {
return Collections::emptyList;
}
for (int i = 0; i < partitionKeys.size(); i++) {
partSpec.put(partitionKeys.get(i), partFields[i]);
}
scan.withPartitionFilter(partSpec);
}
// TODO support range?
if (partition != null) {
scan.withPartitionFilter(Collections.singletonList(partition));
}
if (bucketPredicate != null) {
scan.withBucketFilter(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -159,8 +159,7 @@ public void testReadFilesFromSpecifiedSnapshot() throws Exception {
}

@Test
public void testReadFilesFromNotExistSnapshot() throws Exception {

public void testReadFilesFromNotExistSnapshot() {
filesTable =
(FilesTable)
filesTable.copy(
Expand Down

0 comments on commit 350990a

Please sign in to comment.