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] Expose IndexFile in Split. #3226

Merged
merged 22 commits into from
Apr 26, 2024
Merged
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,7 @@ public class DataSplit implements Split {
@Nullable private List<DeletionFile> dataDeletionFiles;

private List<RawFile> rawFiles = Collections.emptyList();
private List<IndexFile> indexFiles = Collections.emptyList();

public DataSplit() {}

Expand Down Expand Up @@ -115,6 +116,15 @@ public Optional<List<RawFile>> convertToRawFiles() {
}
}

@Override
public Optional<List<IndexFile>> indexFiles() {
if (indexFiles.isEmpty()) {
return Optional.empty();
} else {
return Optional.of(indexFiles);
}
}

@Override
public boolean equals(Object o) {
if (this == o) {
Expand All @@ -131,7 +141,8 @@ public boolean equals(Object o) {
&& Objects.equals(dataFiles, split.dataFiles)
&& Objects.equals(dataDeletionFiles, split.dataDeletionFiles)
&& isStreaming == split.isStreaming
&& Objects.equals(rawFiles, split.rawFiles);
&& Objects.equals(rawFiles, split.rawFiles)
&& Objects.equals(indexFiles, split.indexFiles);
}

@Override
Expand All @@ -144,7 +155,8 @@ public int hashCode() {
dataFiles,
dataDeletionFiles,
isStreaming,
rawFiles);
rawFiles,
indexFiles);
}

private void writeObject(ObjectOutputStream out) throws IOException {
Expand All @@ -165,6 +177,7 @@ private void assign(DataSplit other) {
this.dataDeletionFiles = other.dataDeletionFiles;
this.isStreaming = other.isStreaming;
this.rawFiles = other.rawFiles;
this.indexFiles = other.indexFiles;
}

public void serialize(DataOutputView out) throws IOException {
Expand Down Expand Up @@ -193,6 +206,8 @@ public void serialize(DataOutputView out) throws IOException {
for (RawFile rawFile : rawFiles) {
rawFile.serialize(out);
}

IndexFile.serializeList(out, indexFiles);
}

public static DataSplit deserialize(DataInputView in) throws IOException {
Expand Down Expand Up @@ -225,6 +240,8 @@ public static DataSplit deserialize(DataInputView in) throws IOException {
rawFiles.add(RawFile.deserialize(in));
}

List<IndexFile> indexFiles = IndexFile.deserializeList(in);

DataSplit.Builder builder =
builder()
.withSnapshot(snapshotId)
Expand All @@ -233,7 +250,8 @@ public static DataSplit deserialize(DataInputView in) throws IOException {
.withBeforeFiles(beforeFiles)
.withDataFiles(dataFiles)
.isStreaming(isStreaming)
.rawFiles(rawFiles);
.rawFiles(rawFiles)
.indexFiles(indexFiles);
if (beforeDeletionFiles != null) {
builder.withBeforeDeletionFiles(beforeDeletionFiles);
}
Expand Down Expand Up @@ -297,6 +315,11 @@ public Builder rawFiles(List<RawFile> rawFiles) {
return this;
}

public Builder indexFiles(List<IndexFile> indexFiles) {
this.split.indexFiles = indexFiles;
leaves12138 marked this conversation as resolved.
Show resolved Hide resolved
return this;
}

public DataSplit build() {
checkArgument(split.partition != null);
checkArgument(split.bucket != -1);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,80 @@
/*
* 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.io.DataInputView;
import org.apache.paimon.io.DataOutputView;

import javax.annotation.Nullable;

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

/** Index file for data file. */
public class IndexFile {

private final String path;

public IndexFile(String path) {
this.path = path;
}

leaves12138 marked this conversation as resolved.
Show resolved Hide resolved
public static void serialize(DataOutputView out, @Nullable IndexFile indexFile)
throws IOException {
if (indexFile == null) {
out.write(0);
} else {
out.write(1);
out.writeUTF(indexFile.path);
}
}

public static void serializeList(DataOutputView out, List<IndexFile> files) throws IOException {
out.writeInt(files.size());
for (IndexFile file : files) {
serialize(out, file);
}
}

@Nullable
public static IndexFile deserialize(DataInputView in) throws IOException {
return in.readByte() == 1 ? new IndexFile(in.readUTF()) : null;
}

public static List<IndexFile> deserializeList(DataInputView in) throws IOException {
List<IndexFile> files = new ArrayList<>();
int size = in.readInt();
for (int i = 0; i < size; i++) {
files.add(IndexFile.deserialize(in));
}
return files;
}

@Override
public boolean equals(Object o) {
if (!(o instanceof IndexFile)) {
return false;
}

IndexFile other = (IndexFile) o;
return Objects.equals(path, other.path);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -51,4 +51,14 @@ default Optional<List<RawFile>> convertToRawFiles() {
default Optional<List<DeletionFile>> deletionFiles() {
return Optional.empty();
}

/**
* * Return the index file of the data file, for example, bloom-filter index. All the type of
* indexes and columns will be stored in one single index file.
*
* <p>If there is no corresponding index file, the element will be null.
*/
default Optional<List<IndexFile>> indexFiles() {
return Optional.empty();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@
import org.apache.paimon.schema.TableSchema;
import org.apache.paimon.table.source.DataSplit;
import org.apache.paimon.table.source.DeletionFile;
import org.apache.paimon.table.source.IndexFile;
import org.apache.paimon.table.source.PlanImpl;
import org.apache.paimon.table.source.RawFile;
import org.apache.paimon.table.source.ScanMode;
Expand All @@ -65,6 +66,7 @@
import java.util.stream.Collectors;

import static org.apache.paimon.deletionvectors.DeletionVectorsIndexFile.DELETION_VECTORS_INDEX;
import static org.apache.paimon.io.DataFilePathFactory.INDEX_PATH_SUFFIX;
import static org.apache.paimon.operation.FileStoreScan.Plan.groupByPartFiles;
import static org.apache.paimon.predicate.PredicateBuilder.transformFieldMapping;

Expand Down Expand Up @@ -290,11 +292,13 @@ private List<DataSplit> generateSplits(
: null;
for (SplitGenerator.SplitGroup splitGroup : splitGroups) {
List<DataFileMeta> dataFiles = splitGroup.files;
String bucketPath = pathFactory.bucketPath(partition, bucket).toString();
builder.withDataFiles(dataFiles);
builder.rawFiles(
splitGroup.rawConvertible
? convertToRawFiles(partition, bucket, dataFiles)
? convertToRawFiles(bucketPath, dataFiles)
: Collections.emptyList());
builder.indexFiles(convertToIndexFiles(bucketPath, dataFiles));
if (deletionVectors) {
builder.withDataDeletionFiles(
getDeletionFiles(dataFiles, deletionIndexFile));
Expand Down Expand Up @@ -439,14 +443,35 @@ private List<DeletionFile> getDeletionFiles(
return deletionFiles;
}

private List<RawFile> convertToRawFiles(
BinaryRow partition, int bucket, List<DataFileMeta> dataFiles) {
String bucketPath = pathFactory.bucketPath(partition, bucket).toString();
private List<RawFile> convertToRawFiles(String bucketPath, List<DataFileMeta> dataFiles) {
return dataFiles.stream()
.map(f -> makeRawTableFile(bucketPath, f))
.collect(Collectors.toList());
}

private List<IndexFile> convertToIndexFiles(String bucketPath, List<DataFileMeta> dataFiles) {
return dataFiles.stream()
.map(
file -> {
List<String> exFiles =
file.extraFiles().stream()
.filter(s -> s.endsWith(INDEX_PATH_SUFFIX))
.collect(Collectors.toList());
if (exFiles.size() == 1) {
return new IndexFile(bucketPath + "/" + exFiles.get(0));
} else if (exFiles.size() == 0) {
return null;
} else {
throw new RuntimeException(
"Wrong number of file index for file "
+ file.fileName()
+ " index files: "
+ String.join(",", exFiles));
}
})
.collect(Collectors.toList());
}

private RawFile makeRawTableFile(String bucketPath, DataFileMeta meta) {
return new RawFile(
bucketPath + "/" + meta.fileName(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import org.apache.paimon.data.BinaryString;
import org.apache.paimon.data.GenericRow;
import org.apache.paimon.data.serializer.InternalRowSerializer;
import org.apache.paimon.fileindex.bloomfilter.BloomFilterFileIndex;
import org.apache.paimon.fs.FileIO;
import org.apache.paimon.fs.FileIOFinder;
import org.apache.paimon.fs.Path;
Expand All @@ -38,6 +39,7 @@
import org.apache.paimon.table.sink.StreamTableCommit;
import org.apache.paimon.table.sink.StreamTableWrite;
import org.apache.paimon.table.source.DataSplit;
import org.apache.paimon.table.source.IndexFile;
import org.apache.paimon.table.source.RawFile;
import org.apache.paimon.types.DataType;
import org.apache.paimon.types.DataTypes;
Expand All @@ -55,6 +57,7 @@
import java.util.Map;
import java.util.UUID;

import static org.apache.paimon.io.DataFilePathFactory.INDEX_PATH_SUFFIX;
import static org.assertj.core.api.Assertions.assertThat;

/** Tests for {@link SnapshotReader}. */
Expand Down Expand Up @@ -255,6 +258,82 @@ public void testGetAppendOnlyRawFiles() throws Exception {
commit.close();
}

@Test
public void testGetAppendOnlyIndexFiles() throws Exception {
RowType rowType =
RowType.of(
new DataType[] {DataTypes.INT(), DataTypes.BIGINT()},
new String[] {"k", "v"});
FileStoreTable table =
createFileStoreTable(rowType, Collections.emptyList(), Collections.emptyList());

String commitUser = UUID.randomUUID().toString();
StreamTableWrite write = table.newWrite(commitUser);
StreamTableCommit commit = table.newCommit(commitUser);
SnapshotReader reader = table.newSnapshotReader();

// write one file

write.write(GenericRow.of(11, 1101L));
write.write(GenericRow.of(12, 1201L));
write.write(GenericRow.of(21, 2101L));
write.write(GenericRow.of(22, 2201L));
commit.commit(1, write.prepareCommit(false, 1));

List<DataSplit> dataSplits = reader.read().dataSplits();
assertThat(dataSplits).hasSize(1);
DataSplit dataSplit = dataSplits.get(0);
assertThat(dataSplit.dataFiles()).hasSize(1);
DataFileMeta meta = dataSplit.dataFiles().get(0);
assertThat(dataSplit.indexFiles())
.hasValue(
Collections.singletonList(
new IndexFile(
String.format(
"%s/bucket-0/%s" + INDEX_PATH_SUFFIX,
tablePath,
meta.fileName()))));

// change file schema

write.close();
SchemaManager schemaManager = new SchemaManager(fileIO, tablePath);
schemaManager.commitChanges(SchemaChange.addColumn("v2", DataTypes.STRING()));
table = table.copyWithLatestSchema();
write = table.newWrite(commitUser);

// write another file

write.write(GenericRow.of(11, 1102L, BinaryString.fromString("eleven")));
write.write(GenericRow.of(12, 1202L, BinaryString.fromString("twelve")));
write.write(GenericRow.of(21, 2102L, BinaryString.fromString("twenty-one")));
write.write(GenericRow.of(22, 2202L, BinaryString.fromString("twenty-two")));
commit.commit(2, write.prepareCommit(false, 2));

dataSplits = reader.read().dataSplits();
assertThat(dataSplits).hasSize(1);
dataSplit = dataSplits.get(0);
assertThat(dataSplit.dataFiles()).hasSize(2);
DataFileMeta meta0 = dataSplit.dataFiles().get(0);
DataFileMeta meta1 = dataSplit.dataFiles().get(1);
assertThat(dataSplit.indexFiles())
.hasValue(
Arrays.asList(
new IndexFile(
String.format(
"%s/bucket-0/%s" + INDEX_PATH_SUFFIX,
tablePath,
meta0.fileName())),
new IndexFile(
String.format(
"%s/bucket-0/%s" + INDEX_PATH_SUFFIX,
tablePath,
meta1.fileName()))));

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

private FileStoreTable createFileStoreTable(
RowType rowType, List<String> partitionKeys, List<String> primaryKeys)
throws Exception {
Expand All @@ -265,6 +344,14 @@ private FileStoreTable createFileStoreTable(
Map<String, String> formatPerLevel = new HashMap<>();
formatPerLevel.put("5", "orc");
options.set(CoreOptions.FILE_FORMAT_PER_LEVEL, formatPerLevel);
// test read with extra files
options.set(
CoreOptions.FILE_INDEX
+ "."
+ BloomFilterFileIndex.BLOOM_FILTER
+ "."
+ CoreOptions.COLUMNS,
rowType.getFieldNames().get(0));

SchemaManager schemaManager = new SchemaManager(fileIO, tablePath);
TableSchema tableSchema =
Expand Down
Loading