Skip to content

Commit

Permalink
Switch all places to use DataFileSet/DeleteFileSet
Browse files Browse the repository at this point in the history
  • Loading branch information
nastra committed Sep 26, 2024
1 parent 53ae48f commit 8002680
Show file tree
Hide file tree
Showing 14 changed files with 91 additions and 339 deletions.
139 changes: 1 addition & 138 deletions api/src/main/java/org/apache/iceberg/util/DataFileSet.java
Original file line number Diff line number Diff line change
Expand Up @@ -18,15 +18,8 @@
*/
package org.apache.iceberg.util;

import java.nio.ByteBuffer;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import org.apache.iceberg.DataFile;
import org.apache.iceberg.FileContent;
import org.apache.iceberg.FileFormat;
import org.apache.iceberg.StructLike;
import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
import org.apache.iceberg.relocated.com.google.common.collect.Sets;
import org.apache.iceberg.types.Comparators;
Expand Down Expand Up @@ -63,7 +56,7 @@ protected boolean isInstance(Object obj) {
return obj instanceof DataFile;
}

private static class DataFileWrapper implements Wrapper<DataFile>, DataFile {
private static class DataFileWrapper implements Wrapper<DataFile> {
private DataFile file;

private DataFileWrapper(DataFile file) {
Expand All @@ -85,136 +78,6 @@ public Wrapper<DataFile> set(DataFile dataFile) {
return this;
}

@Override
public Long pos() {
return file.pos();
}

@Override
public int specId() {
return file.specId();
}

@Override
public FileContent content() {
return file.content();
}

@Override
public CharSequence path() {
return file.path();
}

@Override
public String location() {
return file.location();
}

@Override
public FileFormat format() {
return file.format();
}

@Override
public StructLike partition() {
return file.partition();
}

@Override
public long recordCount() {
return file.recordCount();
}

@Override
public long fileSizeInBytes() {
return file.fileSizeInBytes();
}

@Override
public Map<Integer, Long> columnSizes() {
return file.columnSizes();
}

@Override
public Map<Integer, Long> valueCounts() {
return file.valueCounts();
}

@Override
public Map<Integer, Long> nullValueCounts() {
return file.nullValueCounts();
}

@Override
public Map<Integer, Long> nanValueCounts() {
return file.nanValueCounts();
}

@Override
public Map<Integer, ByteBuffer> lowerBounds() {
return file.lowerBounds();
}

@Override
public Map<Integer, ByteBuffer> upperBounds() {
return file.upperBounds();
}

@Override
public ByteBuffer keyMetadata() {
return file.keyMetadata();
}

@Override
public List<Long> splitOffsets() {
return file.splitOffsets();
}

@Override
public List<Integer> equalityFieldIds() {
return file.equalityFieldIds();
}

@Override
public DataFile copy() {
return file.copy();
}

@Override
public DataFile copyWithoutStats() {
return file.copyWithoutStats();
}

@Override
public String manifestLocation() {
return file.manifestLocation();
}

@Override
public Integer sortOrderId() {
return file.sortOrderId();
}

@Override
public Long dataSequenceNumber() {
return file.dataSequenceNumber();
}

@Override
public Long fileSequenceNumber() {
return file.fileSequenceNumber();
}

@Override
public DataFile copyWithStats(Set<Integer> requestedColumnIds) {
return file.copyWithStats(requestedColumnIds);
}

@Override
public DataFile copy(boolean withStats) {
return file.copy(withStats);
}

@Override
public boolean equals(Object o) {
if (this == o) {
Expand Down
139 changes: 1 addition & 138 deletions api/src/main/java/org/apache/iceberg/util/DeleteFileSet.java
Original file line number Diff line number Diff line change
Expand Up @@ -18,15 +18,8 @@
*/
package org.apache.iceberg.util;

import java.nio.ByteBuffer;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import org.apache.iceberg.DeleteFile;
import org.apache.iceberg.FileContent;
import org.apache.iceberg.FileFormat;
import org.apache.iceberg.StructLike;
import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
import org.apache.iceberg.relocated.com.google.common.collect.Sets;
import org.apache.iceberg.types.Comparators;
Expand Down Expand Up @@ -63,7 +56,7 @@ protected boolean isInstance(Object obj) {
return obj instanceof DeleteFile;
}

private static class DeleteFileWrapper implements Wrapper<DeleteFile>, DeleteFile {
private static class DeleteFileWrapper implements Wrapper<DeleteFile> {
private DeleteFile file;

private DeleteFileWrapper(DeleteFile file) {
Expand All @@ -85,136 +78,6 @@ public Wrapper<DeleteFile> set(DeleteFile DeleteFile) {
return this;
}

@Override
public Long pos() {
return file.pos();
}

@Override
public int specId() {
return file.specId();
}

@Override
public FileContent content() {
return file.content();
}

@Override
public CharSequence path() {
return file.path();
}

@Override
public String location() {
return file.location();
}

@Override
public FileFormat format() {
return file.format();
}

@Override
public StructLike partition() {
return file.partition();
}

@Override
public long recordCount() {
return file.recordCount();
}

@Override
public long fileSizeInBytes() {
return file.fileSizeInBytes();
}

@Override
public Map<Integer, Long> columnSizes() {
return file.columnSizes();
}

@Override
public Map<Integer, Long> valueCounts() {
return file.valueCounts();
}

@Override
public Map<Integer, Long> nullValueCounts() {
return file.nullValueCounts();
}

@Override
public Map<Integer, Long> nanValueCounts() {
return file.nanValueCounts();
}

@Override
public Map<Integer, ByteBuffer> lowerBounds() {
return file.lowerBounds();
}

@Override
public Map<Integer, ByteBuffer> upperBounds() {
return file.upperBounds();
}

@Override
public ByteBuffer keyMetadata() {
return file.keyMetadata();
}

@Override
public List<Long> splitOffsets() {
return file.splitOffsets();
}

@Override
public List<Integer> equalityFieldIds() {
return file.equalityFieldIds();
}

@Override
public DeleteFile copy() {
return file.copy();
}

@Override
public DeleteFile copyWithoutStats() {
return file.copyWithoutStats();
}

@Override
public String manifestLocation() {
return file.manifestLocation();
}

@Override
public Integer sortOrderId() {
return file.sortOrderId();
}

@Override
public Long dataSequenceNumber() {
return file.dataSequenceNumber();
}

@Override
public Long fileSequenceNumber() {
return file.fileSequenceNumber();
}

@Override
public DeleteFile copyWithStats(Set<Integer> requestedColumnIds) {
return file.copyWithStats(requestedColumnIds);
}

@Override
public DeleteFile copy(boolean withStats) {
return file.copy(withStats);
}

@Override
public boolean equals(Object o) {
if (this == o) {
Expand Down
4 changes: 2 additions & 2 deletions core/src/main/java/org/apache/iceberg/BaseOverwriteFiles.java
Original file line number Diff line number Diff line change
Expand Up @@ -25,11 +25,11 @@
import org.apache.iceberg.expressions.Projections;
import org.apache.iceberg.expressions.StrictMetricsEvaluator;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.util.ContentFileSet;
import org.apache.iceberg.util.DataFileSet;

public class BaseOverwriteFiles extends MergingSnapshotProducer<OverwriteFiles>
implements OverwriteFiles {
private final ContentFileSet<DataFile> deletedDataFiles = ContentFileSet.empty();
private final DataFileSet deletedDataFiles = DataFileSet.empty();
private boolean validateAddedFilesMatchOverwriteFilter = false;
private Long startingSnapshotId = null;
private Expression conflictDetectionFilter = null;
Expand Down
4 changes: 2 additions & 2 deletions core/src/main/java/org/apache/iceberg/BaseRewriteFiles.java
Original file line number Diff line number Diff line change
Expand Up @@ -21,10 +21,10 @@
import java.util.Set;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
import org.apache.iceberg.util.ContentFileSet;
import org.apache.iceberg.util.DataFileSet;

class BaseRewriteFiles extends MergingSnapshotProducer<RewriteFiles> implements RewriteFiles {
private final ContentFileSet<DataFile> replacedDataFiles = ContentFileSet.empty();
private final DataFileSet replacedDataFiles = DataFileSet.empty();
private Long startingSnapshotId = null;

BaseRewriteFiles(String tableName, TableOperations ops) {
Expand Down
4 changes: 2 additions & 2 deletions core/src/main/java/org/apache/iceberg/FastAppend.java
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.apache.iceberg.util.ContentFileSet;
import org.apache.iceberg.util.DataFileSet;

/**
* {@link AppendFiles Append} implementation that adds a new manifest file for the write.
Expand All @@ -43,7 +43,7 @@ class FastAppend extends SnapshotProducer<AppendFiles> implements AppendFiles {
private final TableOperations ops;
private final PartitionSpec spec;
private final SnapshotSummary.Builder summaryBuilder = SnapshotSummary.builder();
private final ContentFileSet<DataFile> newFiles = ContentFileSet.empty();
private final DataFileSet newFiles = DataFileSet.empty();
private final List<ManifestFile> appendManifests = Lists.newArrayList();
private final List<ManifestFile> rewrittenAppendManifests = Lists.newArrayList();
private List<ManifestFile> newManifests = null;
Expand Down
Loading

0 comments on commit 8002680

Please sign in to comment.