Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[core] Remove file io from splits and Add preferFileIO for Paimon catalog. #2995

Merged
merged 9 commits into from
Mar 13, 2024
Merged
Show file tree
Hide file tree
Changes from 7 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 @@ -41,14 +41,17 @@ public class CatalogContext {

private final Options options;
private final Configuration hadoopConf;
@Nullable private final FileIOLoader preferIOLoader;
@Nullable private final FileIOLoader fallbackIOLoader;

private CatalogContext(
Options options,
@Nullable Configuration hadoopConf,
@Nullable FileIOLoader preferIOLoader,
@Nullable FileIOLoader fallbackIOLoader) {
this.options = checkNotNull(options);
this.hadoopConf = hadoopConf == null ? getHadoopConfiguration(options) : hadoopConf;
this.preferIOLoader = preferIOLoader;
this.fallbackIOLoader = fallbackIOLoader;
}

Expand All @@ -59,20 +62,28 @@ public static CatalogContext create(Path warehouse) {
}

public static CatalogContext create(Options options) {
return new CatalogContext(options, null, null);
return new CatalogContext(options, null, null, null);
}

public static CatalogContext create(Options options, Configuration hadoopConf) {
return new CatalogContext(options, hadoopConf, null);
return new CatalogContext(options, hadoopConf, null, null);
}

public static CatalogContext create(Options options, FileIOLoader fallbackIOLoader) {
return new CatalogContext(options, null, fallbackIOLoader);
return new CatalogContext(options, null, null, fallbackIOLoader);
}

public static CatalogContext create(
Options options, Configuration hadoopConf, FileIOLoader fallbackIOLoader) {
return new CatalogContext(options, hadoopConf, fallbackIOLoader);
Options options, FileIOLoader preferIOLoader, FileIOLoader fallbackIOLoader) {
return new CatalogContext(options, null, preferIOLoader, fallbackIOLoader);
}

public static CatalogContext create(
Options options,
Configuration hadoopConf,
FileIOLoader preferIOLoader,
FileIOLoader fallbackIOLoader) {
return new CatalogContext(options, hadoopConf, preferIOLoader, fallbackIOLoader);
}

public Options options() {
Expand All @@ -84,6 +95,11 @@ public Configuration hadoopConf() {
return hadoopConf;
}

@Nullable
public FileIOLoader preIO() {
leaves12138 marked this conversation as resolved.
Show resolved Hide resolved
return preferIOLoader;
}

@Nullable
public FileIOLoader fallbackIO() {
return fallbackIOLoader;
Expand Down
27 changes: 21 additions & 6 deletions paimon-common/src/main/java/org/apache/paimon/fs/FileIO.java
Original file line number Diff line number Diff line change
Expand Up @@ -316,14 +316,25 @@ static FileIO get(Path path, CatalogContext config) throws IOException {
+ "')");
}

Map<String, FileIOLoader> loaders = discoverLoaders();
FileIOLoader loader = loaders.get(uri.getScheme());
FileIOLoader loader = null;
List<IOException> ioExceptionList = new ArrayList<>();

// load preIO
FileIOLoader preIO = config.preIO();
try {
loader = checkAccess(preIO, path, config);
} catch (IOException ioException) {
ioExceptionList.add(ioException);
}

if (loader == null) {
Map<String, FileIOLoader> loaders = discoverLoaders();
loader = loaders.get(uri.getScheme());
}

// load fallbackIO
FileIOLoader fallbackIO = config.fallbackIO();

List<IOException> ioExceptionList = new ArrayList<>();

if (loader != null) {
Set<String> options =
config.options().keySet().stream()
Expand Down Expand Up @@ -374,6 +385,10 @@ static FileIO get(Path path, CatalogContext config) throws IOException {

if (loader == null) {
String fallbackMsg = "";
String preMsg = "";
if (preIO != null) {
preMsg = " " + preIO.getClass().getSimpleName() + " also cannot access this path.";
leaves12138 marked this conversation as resolved.
Show resolved Hide resolved
}
if (fallbackIO != null) {
fallbackMsg =
" "
Expand All @@ -384,8 +399,8 @@ static FileIO get(Path path, CatalogContext config) throws IOException {
new UnsupportedSchemeException(
String.format(
"Could not find a file io implementation for scheme '%s' in the classpath."
+ "%s Hadoop FileSystem also cannot access this path '%s'.",
uri.getScheme(), fallbackMsg, path));
+ "%s %s Hadoop FileSystem also cannot access this path '%s'.",
uri.getScheme(), preMsg, fallbackMsg, path));
for (IOException ioException : ioExceptionList) {
ex.addSuppressed(ioException);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,12 +20,13 @@

import org.apache.paimon.table.source.snapshot.StartingScanner;

import java.io.Serializable;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;

/** Scanning plan containing snapshot ID and input splits. */
public class DataFilePlan implements TableScan.Plan {
public class DataFilePlan implements TableScan.Plan, Serializable {
leaves12138 marked this conversation as resolved.
Show resolved Hide resolved

private final List<DataSplit> splits;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,8 +25,6 @@
import org.apache.paimon.table.source.snapshot.StartingScanner;
import org.apache.paimon.table.source.snapshot.StartingScanner.ScannedResult;

import javax.annotation.Nullable;

import java.util.ArrayList;
import java.util.List;

Expand Down Expand Up @@ -93,24 +91,7 @@ private StartingScanner.Result applyPushDownLimit(StartingScanner.Result result)
}

SnapshotReader.Plan newPlan =
new SnapshotReader.Plan() {
@Nullable
@Override
public Long watermark() {
return plan.watermark();
}

@Nullable
@Override
public Long snapshotId() {
return plan.snapshotId();
}

@Override
public List<Split> splits() {
return limitedSplits;
}
};
new SimplePlan(plan.watermark(), plan.snapshotId(), limitedSplits);
return new ScannedResult(newPlan);
} else {
return result;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,57 @@
/*
* 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.source;

import org.apache.paimon.table.source.snapshot.SnapshotReader;

import org.jetbrains.annotations.Nullable;

import java.io.Serializable;
import java.util.List;

/** A simple implementation of {@link SnapshotReader.Plan}. */
public class SimplePlan implements SnapshotReader.Plan, Serializable {
leaves12138 marked this conversation as resolved.
Show resolved Hide resolved

private final Long watermark;
private final Long snapshotId;
private final List<Split> splits;

public SimplePlan(Long watermark, Long snapshotId, List<Split> splits) {
this.watermark = watermark;
this.snapshotId = snapshotId;
this.splits = splits;
}

@Nullable
@Override
public Long watermark() {
return watermark;
}

@Nullable
@Override
public Long snapshotId() {
return snapshotId;
}

@Override
public List<Split> splits() {
return splits;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@
import org.apache.paimon.io.DataFileMeta;
import org.apache.paimon.table.source.DataSplit;
import org.apache.paimon.table.source.ScanMode;
import org.apache.paimon.table.source.Split;
import org.apache.paimon.table.source.SimplePlan;
import org.apache.paimon.utils.Pair;
import org.apache.paimon.utils.SnapshotManager;

Expand Down Expand Up @@ -78,23 +78,7 @@ public Result scan(SnapshotReader reader) {
}
}

return StartingScanner.fromPlan(
new SnapshotReader.Plan() {
@Override
public Long watermark() {
return null;
}

@Override
public Long snapshotId() {
return endingSnapshotId;
}

@Override
public List<Split> splits() {
return (List) result;
}
});
return StartingScanner.fromPlan(new SimplePlan(null, endingSnapshotId, (List) result));
}

private List<DataSplit> readSplits(SnapshotReader reader, Snapshot s) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@
import org.apache.paimon.table.source.DeletionFile;
import org.apache.paimon.table.source.RawFile;
import org.apache.paimon.table.source.ScanMode;
import org.apache.paimon.table.source.Split;
import org.apache.paimon.table.source.SimplePlan;
import org.apache.paimon.table.source.SplitGenerator;
import org.apache.paimon.types.RowType;
import org.apache.paimon.utils.FileStorePathFactory;
Expand Down Expand Up @@ -252,24 +252,7 @@ public Plan read() {
scanMode != ScanMode.ALL,
splitGenerator,
files);
return new Plan() {
@Nullable
@Override
public Long watermark() {
return plan.watermark();
}

@Nullable
@Override
public Long snapshotId() {
return plan.snapshotId();
}

@Override
public List<Split> splits() {
return (List) splits;
}
};
return new SimplePlan(plan.watermark(), plan.snapshotId(), (List) splits);
}

private List<DataSplit> generateSplits(
Expand Down Expand Up @@ -401,24 +384,7 @@ private Plan toChangesPlan(
}
}

return new Plan() {
@Nullable
@Override
public Long watermark() {
return plan.watermark();
}

@Nullable
@Override
public Long snapshotId() {
return plan.snapshotId();
}

@Override
public List<Split> splits() {
return (List) splits;
}
};
return new SimplePlan(plan.watermark(), plan.snapshotId(), (List) splits);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -120,7 +120,11 @@ public InnerTableScan withFilter(Predicate predicate) {

@Override
public Plan innerPlan() {
return () -> Collections.singletonList(new AggregationSplit(fileIO, location));
return () ->
Collections.singletonList(
new AggregationSplit(
new SchemaManager(fileIO, location).listAllIds().size(),
location));
}
}

Expand All @@ -129,17 +133,17 @@ private static class AggregationSplit implements Split {

private static final long serialVersionUID = 1L;

private final FileIO fileIO;
Copy link
Contributor

Choose a reason for hiding this comment

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

Can you add a test to verify no fileIO here?

private final long rowCount;
private final Path location;

private AggregationSplit(FileIO fileIO, Path location) {
this.fileIO = fileIO;
private AggregationSplit(long rowCount, Path location) {
this.rowCount = rowCount;
this.location = location;
}

@Override
public long rowCount() {
return new SchemaManager(fileIO, location).listAllIds().size();
return rowCount;
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -122,25 +122,26 @@ public InnerTableScan withFilter(Predicate predicate) {

@Override
public Plan innerPlan() {
return () -> Collections.singletonList(new BranchesSplit(fileIO, location));
FileStoreTable table = FileStoreTableFactory.create(fileIO, location);
long rowCount = table.branchManager().branchCount();
return () -> Collections.singletonList(new BranchesSplit(rowCount, location));
}
}

private static class BranchesSplit implements Split {
private static final long serialVersionUID = 1L;

private final FileIO fileIO;
private final long rowCount;
private final Path location;

private BranchesSplit(FileIO fileIO, Path location) {
this.fileIO = fileIO;
private BranchesSplit(long rowCount, Path location) {
this.rowCount = rowCount;
this.location = location;
}

@Override
public long rowCount() {
FileStoreTable table = FileStoreTableFactory.create(fileIO, location);
return table.branchManager().branchCount();
return rowCount;
}

@Override
Expand Down
Loading
Loading